diff --git a/.gitignore b/.gitignore index 04f8feed0ad70..9d5d1d09dd851 100644 --- a/.gitignore +++ b/.gitignore @@ -1,5 +1,6 @@ dist *classes +*.class target/ build/ build_eclipse/ @@ -33,8 +34,7 @@ Vagrantfile.local config/server-* config/zookeeper-* core/data/* -gradle/wrapper/* -gradlew +gradle/wrapper/*.jar gradlew.bat results diff --git a/README.md b/README.md index 18cb03e988048..0168caecc46da 100644 --- a/README.md +++ b/README.md @@ -2,18 +2,10 @@ Apache Kafka ================= See our [web site](http://kafka.apache.org) for details on the project. -You need to have [Gradle](http://www.gradle.org/installation) and [Java](http://www.oracle.com/technetwork/java/javase/downloads/index.html) installed. - -Kafka requires Gradle 4.5 or higher. +You need to have [Java](http://www.oracle.com/technetwork/java/javase/downloads/index.html) installed. Java 8 should be used for building in order to support both Java 8 and Java 10 at runtime. -### First bootstrap and download the wrapper ### - cd kafka_source_dir - gradle - -Now everything else will work. - ### Build a jar and run it ### ./gradlew jar diff --git a/Vagrantfile b/Vagrantfile index 363607691fd0d..ee08487be66cd 100644 --- a/Vagrantfile +++ b/Vagrantfile @@ -40,8 +40,10 @@ ec2_keypair_file = nil ec2_region = "us-east-1" ec2_az = nil # Uses set by AWS -ec2_ami = "ami-905730e8" +ec2_ami = "ami-29ebb519" ec2_instance_type = "m3.medium" +ec2_spot_instance = ENV['SPOT_INSTANCE'] ? ENV['SPOT_INSTANCE'] == 'true' : true +ec2_spot_max_price = "0.113" # On-demand price for instance type ec2_user = "ubuntu" ec2_instance_name_prefix = "kafka-vagrant" ec2_security_groups = nil @@ -50,6 +52,9 @@ ec2_subnet_id = nil # are running Vagrant from within that VPC as well. ec2_associate_public_ip = nil +jdk_major = '8' +jdk_full = '8u202-linux-x64' + 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") @@ -73,15 +78,6 @@ Vagrant.configure(VAGRANTFILE_API_VERSION) do |config| 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-jdk8" => { cache_dir: "/tmp/oracle-jdk8-installer-cache" }, - } end end @@ -133,6 +129,10 @@ Vagrant.configure(VAGRANTFILE_API_VERSION) do |config| else aws.associate_public_ip = ec2_associate_public_ip end + aws.region_config ec2_region do |region| + region.spot_instance = ec2_spot_instance + region.spot_max_price = ec2_spot_max_price + end # Exclude some directories that can grow very large from syncing override.vm.synced_folder ".", "/vagrant", type: "rsync", rsync__exclude: ['.git', 'core/data/', 'logs/', 'tests/results/', 'results/'] @@ -163,7 +163,7 @@ Vagrant.configure(VAGRANTFILE_API_VERSION) do |config| name_node(zookeeper, name, ec2_instance_name_prefix) ip_address = "192.168.50." + (10 + i).to_s assign_local_ip(zookeeper, ip_address) - zookeeper.vm.provision "shell", path: "vagrant/base.sh" + zookeeper.vm.provision "shell", path: "vagrant/base.sh", env: {"JDK_MAJOR" => jdk_major, "JDK_FULL" => jdk_full} 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 @@ -180,7 +180,7 @@ Vagrant.configure(VAGRANTFILE_API_VERSION) do |config| # 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" + broker.vm.provision "shell", path: "vagrant/base.sh", env: {"JDK_MAJOR" => jdk_major, "JDK_FULL" => jdk_full} 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 @@ -192,7 +192,7 @@ Vagrant.configure(VAGRANTFILE_API_VERSION) do |config| name_node(worker, name, ec2_instance_name_prefix) ip_address = "192.168.50." + (100 + i).to_s assign_local_ip(worker, ip_address) - worker.vm.provision "shell", path: "vagrant/base.sh" + worker.vm.provision "shell", path: "vagrant/base.sh", env: {"JDK_MAJOR" => jdk_major, "JDK_FULL" => jdk_full} end } diff --git a/bin/kafka-run-class.sh b/bin/kafka-run-class.sh index 354e10650ee2d..eed39557f9c4a 100755 --- a/bin/kafka-run-class.sh +++ b/bin/kafka-run-class.sh @@ -107,7 +107,7 @@ else for file in "$base_dir"/streams/upgrade-system-tests-$SHORT_VERSION_NO_DOTS/build/libs/kafka-streams-upgrade-system-tests*.jar; do if should_include_file "$file"; then - CLASSPATH="$CLASSPATH":"$file" + CLASSPATH="$file":"$CLASSPATH" fi done fi @@ -129,7 +129,7 @@ do CLASSPATH="$CLASSPATH:$dir/*" done -for cc_pkg in "api" "transforms" "runtime" "file" "json" "tools" +for cc_pkg in "api" "transforms" "runtime" "file" "json" "tools" "basic-auth-extension" do for file in "$base_dir"/connect/${cc_pkg}/build/libs/connect-${cc_pkg}*.jar; do diff --git a/bin/windows/kafka-run-class.bat b/bin/windows/kafka-run-class.bat index 1dfd9a5abfede..a21171b65d272 100755 --- a/bin/windows/kafka-run-class.bat +++ b/bin/windows/kafka-run-class.bat @@ -111,7 +111,7 @@ IF ["%JMX_PORT%"] NEQ [""] ( rem Log directory to use IF ["%LOG_DIR%"] EQU [""] ( - set LOG_DIR="%BASE_DIR~%/logs" + set LOG_DIR="%BASE_DIR%/logs" ) rem Log4j settings diff --git a/bin/windows/kafka-simple-consumer-shell.bat b/bin/windows/kafka-simple-consumer-shell.bat deleted file mode 100644 index 8836128b8b05a..0000000000000 --- a/bin/windows/kafka-simple-consumer-shell.bat +++ /dev/null @@ -1,17 +0,0 @@ -@echo off -rem Licensed to the Apache Software Foundation (ASF) under one or more -rem contributor license agreements. See the NOTICE file distributed with -rem this work for additional information regarding copyright ownership. -rem The ASF licenses this file to You under the Apache License, Version 2.0 -rem (the "License"); you may not use this file except in compliance with -rem the License. You may obtain a copy of the License at -rem -rem http://www.apache.org/licenses/LICENSE-2.0 -rem -rem Unless required by applicable law or agreed to in writing, software -rem distributed under the License is distributed on an "AS IS" BASIS, -rem WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -rem See the License for the specific language governing permissions and -rem limitations under the License. - -"%~dp0kafka-run-class.bat" kafka.tools.SimpleConsumerShell %* diff --git a/build.gradle b/build.gradle index 4f3fd770b78a3..d15c22b790073 100644 --- a/build.gradle +++ b/build.gradle @@ -26,9 +26,17 @@ buildscript { // For Apache Rat plugin to ignore non-Git files classpath "org.ajoberstar:grgit:1.9.3" classpath 'com.github.ben-manes:gradle-versions-plugin:0.17.0' - classpath 'org.scoverage:gradle-scoverage:2.3.0' classpath 'com.github.jengelman.gradle.plugins:shadow:2.0.4' classpath 'org.owasp:dependency-check-gradle:3.1.2' + classpath "com.diffplug.spotless:spotless-plugin-gradle:3.10.0" + } +} + +apply plugin: "com.diffplug.gradle.spotless" +spotless { + scala { + target 'streams/**/*.scala' + scalafmt('1.5.1').configFile('checkstyle/.scalafmt.conf') } } @@ -76,7 +84,7 @@ allprojects { } ext { - gradleVersion = "4.7" + gradleVersion = "4.8.1" minJavaVersion = "8" buildVersionFileName = "kafka-version.properties" @@ -393,13 +401,12 @@ subprojects { } } - def coverageGen = it.path == ':core' ? 'reportScoverage' : 'jacocoTestReport' - task reportCoverage(dependsOn: [coverageGen]) - + def coverageGen = it.path == ':core' ? [] : ['jacocoTestReport'] + task reportCoverage(dependsOn: coverageGen) } gradle.taskGraph.whenReady { taskGraph -> - taskGraph.getAllTasks().findAll { it.name.contains('findbugsScoverage') || it.name.contains('findbugsTest') }.each { task -> + taskGraph.getAllTasks().findAll { it.name.contains('findbugsTest') }.each { task -> task.enabled = false } } @@ -464,28 +471,28 @@ task reportCoverage(dependsOn: ['jacocoRootReport', 'core:reportCoverage']) for ( sv in availableScalaVersions ) { String taskSuffix = sv.replaceAll("\\.", "_") - tasks.create(name: "jar_core_${taskSuffix}", type: GradleBuild) { + tasks.create(name: "jarScala_${taskSuffix}", type: GradleBuild) { startParameter = project.getGradle().getStartParameter().newInstance() startParameter.projectProperties += [scalaVersion: "${sv}"] - tasks = ['core:jar'] + tasks = ['core:jar', 'streams:streams-scala:jar'] } - tasks.create(name: "test_core_${taskSuffix}", type: GradleBuild) { + tasks.create(name: "testScala_${taskSuffix}", type: GradleBuild) { startParameter = project.getGradle().getStartParameter().newInstance() startParameter.projectProperties += [scalaVersion: "${sv}"] - tasks = ['core:test'] + tasks = ['core:test', 'streams:streams-scala:test'] } tasks.create(name: "srcJar_${taskSuffix}", type: GradleBuild) { startParameter = project.getGradle().getStartParameter().newInstance() startParameter.projectProperties += [scalaVersion: "${sv}"] - tasks = ['core:srcJar'] + tasks = ['core:srcJar', 'streams:streams-scala:srcJar'] } tasks.create(name: "docsJar_${taskSuffix}", type: GradleBuild) { startParameter = project.getGradle().getStartParameter().newInstance() startParameter.projectProperties += [scalaVersion: "${sv}"] - tasks = ['core:docsJar'] + tasks = ['core:docsJar', 'streams:streams-scala:docsJar'] } tasks.create(name: "install_${taskSuffix}", type: GradleBuild) { @@ -500,14 +507,14 @@ for ( sv in availableScalaVersions ) { tasks = ['releaseTarGz'] } - tasks.create(name: "uploadCoreArchives_${taskSuffix}", type: GradleBuild) { + tasks.create(name: "uploadScalaArchives_${taskSuffix}", type: GradleBuild) { startParameter = project.getGradle().getStartParameter().newInstance() startParameter.projectProperties += [scalaVersion: "${sv}"] - tasks = ['core:uploadArchives'] + tasks = ['core:uploadArchives', 'streams:streams-scala:uploadArchives'] } } -def connectPkgs = ['connect:api', 'connect:runtime', 'connect:transforms', 'connect:json', 'connect:file'] +def connectPkgs = ['connect:api', 'connect:runtime', 'connect:transforms', 'connect:json', 'connect:file', 'connect:basic-auth-extension'] def pkgs = ['clients', 'examples', 'log4j-appender', 'tools', 'streams', 'streams:streams-scala', 'streams:test-utils', 'streams:examples'] + connectPkgs /** Create one task per default Scala version */ @@ -516,26 +523,25 @@ def withDefScalaVersions(taskName) { } tasks.create(name: "jarConnect", dependsOn: connectPkgs.collect { it + ":jar" }) {} -tasks.create(name: "jarAll", dependsOn: withDefScalaVersions('jar_core') + pkgs.collect { it + ":jar" }) { } +tasks.create(name: "jarAll", dependsOn: withDefScalaVersions('jarScala') + pkgs.collect { it + ":jar" }) { } tasks.create(name: "srcJarAll", dependsOn: withDefScalaVersions('srcJar') + pkgs.collect { it + ":srcJar" }) { } tasks.create(name: "docsJarAll", dependsOn: withDefScalaVersions('docsJar') + pkgs.collect { it + ":docsJar" }) { } tasks.create(name: "testConnect", dependsOn: connectPkgs.collect { it + ":test" }) {} -tasks.create(name: "testAll", dependsOn: withDefScalaVersions('test_core') + pkgs.collect { it + ":test" }) { } +tasks.create(name: "testAll", dependsOn: withDefScalaVersions('testScala') + pkgs.collect { it + ":test" }) { } tasks.create(name: "installAll", dependsOn: withDefScalaVersions('install') + pkgs.collect { it + ":install" }) { } tasks.create(name: "releaseTarGzAll", dependsOn: withDefScalaVersions('releaseTarGz')) { } -tasks.create(name: "uploadArchivesAll", dependsOn: withDefScalaVersions('uploadCoreArchives') + pkgs.collect { it + ":uploadArchives" }) { } +tasks.create(name: "uploadArchivesAll", dependsOn: withDefScalaVersions('uploadScalaArchives') + pkgs.collect { it + ":uploadArchives" }) { } project(':core') { println "Building project 'core' with Scala version ${versions.scala}" apply plugin: 'scala' - apply plugin: "org.scoverage" archivesBaseName = "kafka_${versions.baseScala}" dependencies { @@ -577,20 +583,8 @@ project(':core') { testCompile libs.scalatest testCompile libs.slf4jlog4j testCompile libs.jfreechart - - scoverage libs.scoveragePlugin - scoverage libs.scoverageRuntime } - scoverage { - reportDir = file("${rootProject.buildDir}/scoverage") - highlighting = false - } - checkScoverage { - minimumRate = 0.0 - } - checkScoverage.shouldRunAfter('test') - configurations { // manually excludes some unnecessary dependencies compile.exclude module: 'javax' @@ -728,6 +722,8 @@ project(':core') { from(project(':connect:json').configurations.runtime) { into("libs/") } from(project(':connect:file').jar) { into("libs/") } from(project(':connect:file').configurations.runtime) { into("libs/") } + from(project(':connect:basic-auth-extension').jar) { into("libs/") } + from(project(':connect:basic-auth-extension').configurations.runtime) { into("libs/") } from(project(':streams').jar) { into("libs/") } from(project(':streams').configurations.runtime) { into("libs/") } from(project(':streams:streams-scala').jar) { into("libs/") } @@ -783,11 +779,20 @@ project(':examples') { project(':clients') { archivesBaseName = "kafka-clients" + configurations { + jacksonDatabindConfig + } + + // add jacksonDatabindConfig as provided scope config with high priority (1000) + conf2ScopeMappings.addMapping(1000, configurations.jacksonDatabindConfig, "provided") + dependencies { compile libs.lz4 compile libs.snappy compile libs.slf4jApi - compile libs.jacksonDatabind // for SASL/OAUTHBEARER bearer token parsing + compileOnly libs.jacksonDatabind // for SASL/OAUTHBEARER bearer token parsing + + jacksonDatabindConfig libs.jacksonDatabind // to publish as provided scope dependency. testCompile libs.bcpkix testCompile libs.junit @@ -796,6 +801,7 @@ project(':clients') { testCompile libs.powermockEasymock testRuntime libs.slf4jlog4j + testRuntime libs.jacksonDatabind } task determineCommitId { @@ -856,10 +862,14 @@ project(':clients') { include "**/org/apache/kafka/common/resource/*" include "**/org/apache/kafka/common/serialization/*" include "**/org/apache/kafka/common/config/*" + include "**/org/apache/kafka/common/config/provider/*" include "**/org/apache/kafka/common/security/auth/*" - include "**/org/apache/kafka/server/policy/*" + include "**/org/apache/kafka/common/security/plain/*" + include "**/org/apache/kafka/common/security/scram/*" include "**/org/apache/kafka/common/security/token/delegation/*" include "**/org/apache/kafka/common/security/oauthbearer/*" + include "**/org/apache/kafka/server/policy/*" + include "**/org/apache/kafka/server/quota/*" } } @@ -918,7 +928,13 @@ project(':streams') { dependencies { compile project(':clients') - compile project(':connect:json') // this dependency should be removed after we unify data API + + // this dependency should be removed after we unify data API + compile(project(':connect:json')) { + // this transitive dependency is not used in Streams, and it breaks SBT builds + exclude module: 'javax.ws.rs-api' + } + compile libs.slf4jApi compile libs.rocksDBJni @@ -983,9 +999,11 @@ project(':streams:streams-scala') { testCompile project(':core').sourceSets.test.output testCompile project(':streams').sourceSets.test.output testCompile project(':clients').sourceSets.test.output + testCompile project(':streams:test-utils') testCompile libs.junit testCompile libs.scalatest + testCompile libs.easymock testRuntime libs.slf4jlog4j } @@ -1017,6 +1035,7 @@ project(':streams:test-utils') { testCompile project(':clients').sourceSets.test.output testCompile libs.junit + testCompile libs.easymock testRuntime libs.slf4jlog4j } @@ -1156,7 +1175,7 @@ project(':jmh-benchmarks') { compile project(':clients') compile project(':streams') compile libs.jmhCore - compile libs.jmhGeneratorAnnProcess + annotationProcessor libs.jmhGeneratorAnnProcess compile libs.jmhCoreBenchmarks } @@ -1207,7 +1226,7 @@ project(':connect:api') { dependencies { compile project(':clients') compile libs.slf4jApi - compile libs.jerseyContainerServlet + compile libs.jaxrsApi testCompile libs.junit @@ -1344,8 +1363,11 @@ project(':connect:runtime') { testCompile libs.junit testCompile libs.powermockJunit4 testCompile libs.powermockEasymock + testCompile libs.httpclient testCompile project(':clients').sourceSets.test.output + testCompile project(':core') + testCompile project(':core').sourceSets.test.output testRuntime libs.slf4jlog4j } @@ -1447,6 +1469,7 @@ project(':connect:basic-auth-extension') { testCompile project(':clients').sourceSets.test.output testRuntime libs.slf4jlog4j + testRuntime libs.jerseyContainerServlet } javadoc { diff --git a/checkstyle/.scalafmt.conf b/checkstyle/.scalafmt.conf new file mode 100644 index 0000000000000..057e3b930962e --- /dev/null +++ b/checkstyle/.scalafmt.conf @@ -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. +docstrings = JavaDoc +maxColumn = 120 +continuationIndent.defnSite = 2 +assumeStandardLibraryStripMargin = true +danglingParentheses = true +rewrite.rules = [SortImports, RedundantBraces, RedundantParens, SortModifiers] \ No newline at end of file diff --git a/checkstyle/import-control-core.xml b/checkstyle/import-control-core.xml index 48659867000f8..f8b1c46ff384c 100644 --- a/checkstyle/import-control-core.xml +++ b/checkstyle/import-control-core.xml @@ -38,26 +38,9 @@ - - - - - - - - - - - - - - - - - @@ -71,9 +54,6 @@ - - - diff --git a/checkstyle/import-control.xml b/checkstyle/import-control.xml index 106ad0a8c70a4..290a2e35105ca 100644 --- a/checkstyle/import-control.xml +++ b/checkstyle/import-control.xml @@ -330,16 +330,16 @@ + + - - @@ -357,6 +357,21 @@ + + + + + + + + + + + + + + + diff --git a/checkstyle/suppressions.xml b/checkstyle/suppressions.xml index 5bf69b6b65f9b..8099324fcd6a4 100644 --- a/checkstyle/suppressions.xml +++ b/checkstyle/suppressions.xml @@ -70,10 +70,10 @@ files="MockAdminClient.java"/> + files="RequestResponseTest.java|FetcherTest.java"/> + files="MemoryRecordsTest|MetricsTest"/> parseAndValidateAddresses(List url return addresses; } - public static void closeQuietly(Closeable c, String name, AtomicReference firstException) { - if (c != null) { - try { - c.close(); - } catch (Throwable t) { - firstException.compareAndSet(null, t); - log.error("Failed to close " + name, t); - } - } - } - /** * @param config client configs * @return configured ChannelBuilder based on the configs. 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 5caee2d4c87d8..5b7ba611714f1 100644 --- a/clients/src/main/java/org/apache/kafka/clients/InFlightRequests.java +++ b/clients/src/main/java/org/apache/kafka/clients/InFlightRequests.java @@ -17,11 +17,11 @@ package org.apache.kafka.clients; import java.util.ArrayDeque; +import java.util.ArrayList; import java.util.Collections; import java.util.Deque; import java.util.HashMap; import java.util.Iterator; -import java.util.LinkedList; import java.util.List; import java.util.Map; import java.util.concurrent.atomic.AtomicInteger; @@ -162,25 +162,28 @@ public Iterator iterator() { } } + private Boolean hasExpiredRequest(long now, Deque deque) { + for (NetworkClient.InFlightRequest request : deque) { + long timeSinceSend = Math.max(0, now - request.sendTimeMs); + if (timeSinceSend > request.requestTimeoutMs) + return true; + } + return false; + } + /** * Returns a list of nodes with pending in-flight request, that need to be timed out * * @param now current time in milliseconds - * @param requestTimeoutMs max time to wait for the request to be completed * @return list of nodes */ - public List getNodesWithTimedOutRequests(long now, int requestTimeoutMs) { - List nodeIds = new LinkedList<>(); + public List nodesWithTimedOutRequests(long now) { + List nodeIds = new ArrayList<>(); for (Map.Entry> requestEntry : requests.entrySet()) { String nodeId = requestEntry.getKey(); Deque deque = requestEntry.getValue(); - - if (!deque.isEmpty()) { - NetworkClient.InFlightRequest request = deque.peekLast(); - long timeSinceSend = now - request.sendTimeMs; - if (timeSinceSend > requestTimeoutMs) - nodeIds.add(nodeId); - } + if (hasExpiredRequest(now, deque)) + 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 49bf3a3eab97b..18a7eefe2022b 100644 --- a/clients/src/main/java/org/apache/kafka/clients/KafkaClient.java +++ b/clients/src/main/java/org/apache/kafka/clients/KafkaClient.java @@ -185,9 +185,32 @@ ClientRequest newClientRequest(String nodeId, AbstractRequest.Builder request * @param requestBuilder the request builder to use * @param createdTimeMs the time in milliseconds to use as the creation time of the request * @param expectResponse true iff we expect a response + * @param requestTimeoutMs Upper bound time in milliseconds to await a response before disconnecting the socket and + * cancelling the request. The request may get cancelled sooner if the socket disconnects + * for any reason including if another pending request to the same node timed out first. * @param callback the callback to invoke when we get a response */ - ClientRequest newClientRequest(String nodeId, AbstractRequest.Builder requestBuilder, long createdTimeMs, - boolean expectResponse, RequestCompletionHandler callback); + ClientRequest newClientRequest(String nodeId, + AbstractRequest.Builder requestBuilder, + long createdTimeMs, + boolean expectResponse, + int requestTimeoutMs, + RequestCompletionHandler callback); + + + + /** + * Initiates shutdown of this client. This method may be invoked from another thread while this + * client is being polled. No further requests may be sent using the client. The current poll() + * will be terminated using wakeup(). The client should be explicitly shutdown using {@link #close()} + * after poll returns. Note that {@link #close()} should not be invoked concurrently while polling. + */ + void initiateClose(); + + /** + * Returns true if the client is still active. Returns false if {@link #initiateClose()} or {@link #close()} + * was invoked for this client. + */ + boolean active(); } diff --git a/clients/src/main/java/org/apache/kafka/clients/ManualMetadataUpdater.java b/clients/src/main/java/org/apache/kafka/clients/ManualMetadataUpdater.java index 8252cf3a9cd12..ec007a6966879 100644 --- a/clients/src/main/java/org/apache/kafka/clients/ManualMetadataUpdater.java +++ b/clients/src/main/java/org/apache/kafka/clients/ManualMetadataUpdater.java @@ -89,4 +89,8 @@ public void handleCompletedMetadataResponse(RequestHeader requestHeader, long no public void requestUpdate() { // Do nothing } + + @Override + public void close() { + } } diff --git a/clients/src/main/java/org/apache/kafka/clients/Metadata.java b/clients/src/main/java/org/apache/kafka/clients/Metadata.java index b1da9de8ac110..17d98395df035 100644 --- a/clients/src/main/java/org/apache/kafka/clients/Metadata.java +++ b/clients/src/main/java/org/apache/kafka/clients/Metadata.java @@ -17,6 +17,7 @@ package org.apache.kafka.clients; import org.apache.kafka.common.Cluster; +import org.apache.kafka.common.KafkaException; import org.apache.kafka.common.internals.ClusterResourceListeners; import org.apache.kafka.common.Node; import org.apache.kafka.common.PartitionInfo; @@ -25,6 +26,7 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import java.io.Closeable; import java.util.ArrayList; import java.util.Collection; import java.util.Collections; @@ -48,7 +50,7 @@ * is removed from the metadata refresh set after an update. Consumers disable topic expiry since they explicitly * manage topics while producers rely on topic expiry to limit the refresh set. */ -public final class Metadata { +public final class Metadata implements Closeable { private static final Logger log = LoggerFactory.getLogger(Metadata.class); @@ -70,6 +72,7 @@ public final class Metadata { private boolean needMetadataForAllTopics; private final boolean allowAutoTopicCreation; private final boolean topicExpiryEnabled; + private boolean isClosed; public Metadata(long refreshBackoffMs, long metadataExpireMs, boolean allowAutoTopicCreation) { this(refreshBackoffMs, metadataExpireMs, allowAutoTopicCreation, false, new ClusterResourceListeners()); @@ -100,6 +103,7 @@ public Metadata(long refreshBackoffMs, long metadataExpireMs, boolean allowAutoT this.listeners = new ArrayList<>(); this.clusterResourceListeners = clusterResourceListeners; this.needMetadataForAllTopics = false; + this.isClosed = false; } /** @@ -120,15 +124,27 @@ public synchronized void add(String topic) { } } + /** + * Return the next time when the current cluster info can be updated (i.e., backoff time has elapsed). + * + * @param nowMs current time in ms + * @return remaining time in ms till the cluster info can be updated again + */ + public synchronized long timeToAllowUpdate(long nowMs) { + return Math.max(this.lastRefreshMs + this.refreshBackoffMs - nowMs, 0); + } + /** * 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 + * + * @param nowMs current time in ms + * @return remaining time in ms till updating the cluster info */ public synchronized long timeToNextUpdate(long nowMs) { long timeToExpire = needUpdate ? 0 : Math.max(this.lastSuccessfulRefreshMs + this.metadataExpireMs - nowMs, 0); - long timeToAllowUpdate = this.lastRefreshMs + this.refreshBackoffMs - nowMs; - return Math.max(timeToExpire, timeToAllowUpdate); + return Math.max(timeToExpire, timeToAllowUpdate(nowMs)); } /** @@ -164,12 +180,12 @@ public synchronized AuthenticationException getAndClearAuthenticationException() * Wait for metadata update until the current version is larger than the last version we know of */ public synchronized void awaitUpdate(final int lastVersion, final long maxWaitMs) throws InterruptedException { - if (maxWaitMs < 0) { + if (maxWaitMs < 0) throw new IllegalArgumentException("Max time to wait for metadata updates should not be < 0 milliseconds"); - } + long begin = System.currentTimeMillis(); long remainingWaitMs = maxWaitMs; - while (this.version <= lastVersion) { + while ((this.version <= lastVersion) && !isClosed()) { AuthenticationException ex = getAndClearAuthenticationException(); if (ex != null) throw ex; @@ -180,6 +196,8 @@ public synchronized void awaitUpdate(final int lastVersion, final long maxWaitMs throw new TimeoutException("Failed to update metadata after " + maxWaitMs + " ms."); remainingWaitMs = maxWaitMs - elapsed; } + if (isClosed()) + throw new KafkaException("Requested metadata update after close"); } /** @@ -224,6 +242,8 @@ public synchronized boolean containsTopic(String topic) { */ public synchronized void update(Cluster newCluster, Set unavailableTopics, long now) { Objects.requireNonNull(newCluster, "cluster should not be null"); + if (isClosed()) + throw new IllegalStateException("Update requested after metadata close"); this.needUpdate = false; this.lastRefreshMs = now; @@ -331,6 +351,25 @@ public synchronized void removeListener(Listener listener) { this.listeners.remove(listener); } + /** + * "Close" this metadata instance to indicate that metadata updates are no longer possible. This is typically used + * when the thread responsible for performing metadata updates is exiting and needs a way to relay this information + * to any other thread(s) that could potentially wait on metadata update to come through. + */ + @Override + public synchronized void close() { + this.isClosed = true; + this.notifyAll(); + } + + /** + * Check if this metadata instance has been closed. See {@link #close()} for more information. + * @return True if this instance has been closed; false otherwise + */ + public synchronized boolean isClosed() { + return this.isClosed; + } + /** * MetadataUpdate Listener */ diff --git a/clients/src/main/java/org/apache/kafka/clients/MetadataUpdater.java b/clients/src/main/java/org/apache/kafka/clients/MetadataUpdater.java index 09ed995d14c3e..de765db5a8db3 100644 --- a/clients/src/main/java/org/apache/kafka/clients/MetadataUpdater.java +++ b/clients/src/main/java/org/apache/kafka/clients/MetadataUpdater.java @@ -21,6 +21,7 @@ import org.apache.kafka.common.requests.MetadataResponse; import org.apache.kafka.common.requests.RequestHeader; +import java.io.Closeable; import java.util.List; /** @@ -29,7 +30,7 @@ *

* This class is not thread-safe! */ -public interface MetadataUpdater { +public interface MetadataUpdater extends Closeable { /** * Gets the current cluster info without blocking. @@ -82,4 +83,10 @@ public interface MetadataUpdater { * start of the update if possible (see `maybeUpdate` for more information). */ void requestUpdate(); + + /** + * Close this updater. + */ + @Override + void close(); } 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 7c87277eb8f0a..4f24b54f76eb1 100644 --- a/clients/src/main/java/org/apache/kafka/clients/NetworkClient.java +++ b/clients/src/main/java/org/apache/kafka/clients/NetworkClient.java @@ -18,7 +18,9 @@ import org.apache.kafka.common.Cluster; import org.apache.kafka.common.Node; +import org.apache.kafka.common.TopicPartition; import org.apache.kafka.common.errors.AuthenticationException; +import org.apache.kafka.common.errors.DisconnectException; import org.apache.kafka.common.errors.UnsupportedVersionException; import org.apache.kafka.common.metrics.Sensor; import org.apache.kafka.common.network.ChannelState; @@ -52,6 +54,8 @@ import java.util.List; import java.util.Map; import java.util.Random; +import java.util.concurrent.atomic.AtomicReference; +import java.util.stream.Collectors; /** * A network client for asynchronous request/response network i/o. This is an internal class used to implement the @@ -61,6 +65,12 @@ */ public class NetworkClient implements KafkaClient { + private enum State { + ACTIVE, + CLOSING, + CLOSED + } + private final Logger log; /* the selector used to perform network i/o */ @@ -88,8 +98,8 @@ public class NetworkClient implements KafkaClient { /* the current correlation id to use when sending requests to servers */ private int correlation; - /* max time in ms for the producer to wait for acknowledgement from server*/ - private final int requestTimeoutMs; + /* default timeout for individual requests to await acknowledgement from servers */ + private final int defaultRequestTimeoutMs; /* time in ms to wait before retrying to create connection to a server */ private final long reconnectBackoffMs; @@ -109,6 +119,8 @@ public class NetworkClient implements KafkaClient { private final Sensor throttleTimeSensor; + private final AtomicReference state; + public NetworkClient(Selectable selector, Metadata metadata, String clientId, @@ -117,15 +129,26 @@ public NetworkClient(Selectable selector, long reconnectBackoffMax, int socketSendBuffer, int socketReceiveBuffer, - int requestTimeoutMs, + int defaultRequestTimeoutMs, Time time, boolean discoverBrokerVersions, ApiVersions apiVersions, LogContext logContext) { - this(null, metadata, selector, clientId, maxInFlightRequestsPerConnection, - reconnectBackoffMs, reconnectBackoffMax, - socketSendBuffer, socketReceiveBuffer, requestTimeoutMs, time, - discoverBrokerVersions, apiVersions, null, logContext); + this(null, + metadata, + selector, + clientId, + maxInFlightRequestsPerConnection, + reconnectBackoffMs, + reconnectBackoffMax, + socketSendBuffer, + socketReceiveBuffer, + defaultRequestTimeoutMs, + time, + discoverBrokerVersions, + apiVersions, + null, + logContext); } public NetworkClient(Selectable selector, @@ -136,16 +159,27 @@ public NetworkClient(Selectable selector, long reconnectBackoffMax, int socketSendBuffer, int socketReceiveBuffer, - int requestTimeoutMs, + int defaultRequestTimeoutMs, Time time, boolean discoverBrokerVersions, ApiVersions apiVersions, Sensor throttleTimeSensor, LogContext logContext) { - this(null, metadata, selector, clientId, maxInFlightRequestsPerConnection, - reconnectBackoffMs, reconnectBackoffMax, - socketSendBuffer, socketReceiveBuffer, requestTimeoutMs, time, - discoverBrokerVersions, apiVersions, throttleTimeSensor, logContext); + this(null, + metadata, + selector, + clientId, + maxInFlightRequestsPerConnection, + reconnectBackoffMs, + reconnectBackoffMax, + socketSendBuffer, + socketReceiveBuffer, + defaultRequestTimeoutMs, + time, + discoverBrokerVersions, + apiVersions, + throttleTimeSensor, + logContext); } public NetworkClient(Selectable selector, @@ -156,15 +190,26 @@ public NetworkClient(Selectable selector, long reconnectBackoffMax, int socketSendBuffer, int socketReceiveBuffer, - int requestTimeoutMs, + int defaultRequestTimeoutMs, Time time, boolean discoverBrokerVersions, ApiVersions apiVersions, LogContext logContext) { - this(metadataUpdater, null, selector, clientId, maxInFlightRequestsPerConnection, - reconnectBackoffMs, reconnectBackoffMax, - socketSendBuffer, socketReceiveBuffer, requestTimeoutMs, time, - discoverBrokerVersions, apiVersions, null, logContext); + this(metadataUpdater, + null, + selector, + clientId, + maxInFlightRequestsPerConnection, + reconnectBackoffMs, + reconnectBackoffMax, + socketSendBuffer, + socketReceiveBuffer, + defaultRequestTimeoutMs, + time, + discoverBrokerVersions, + apiVersions, + null, + logContext); } private NetworkClient(MetadataUpdater metadataUpdater, @@ -176,7 +221,7 @@ private NetworkClient(MetadataUpdater metadataUpdater, long reconnectBackoffMax, int socketSendBuffer, int socketReceiveBuffer, - int requestTimeoutMs, + int defaultRequestTimeoutMs, Time time, boolean discoverBrokerVersions, ApiVersions apiVersions, @@ -201,13 +246,14 @@ private NetworkClient(MetadataUpdater metadataUpdater, this.socketReceiveBuffer = socketReceiveBuffer; this.correlation = 0; this.randOffset = new Random(); - this.requestTimeoutMs = requestTimeoutMs; + this.defaultRequestTimeoutMs = defaultRequestTimeoutMs; this.reconnectBackoffMs = reconnectBackoffMs; this.time = time; this.discoverBrokerVersions = discoverBrokerVersions; this.apiVersions = apiVersions; this.throttleTimeSensor = throttleTimeSensor; this.log = logContext.logger(NetworkClient.class); + this.state = new AtomicReference<>(State.ACTIVE); } /** @@ -383,6 +429,7 @@ private void sendInternalMetadataRequest(MetadataRequest.Builder builder, } private void doSend(ClientRequest clientRequest, boolean isInternalRequest, long now) { + ensureActive(); String nodeId = clientRequest.destination(); if (!isInternalRequest) { // If this request came from outside the NetworkClient, validate @@ -426,31 +473,28 @@ private void doSend(ClientRequest clientRequest, boolean isInternalRequest, long } private void doSend(ClientRequest clientRequest, boolean isInternalRequest, long now, AbstractRequest request) { - String nodeId = clientRequest.destination(); + String destination = clientRequest.destination(); RequestHeader header = clientRequest.makeHeader(request.version()); if (log.isDebugEnabled()) { int latestClientVersion = clientRequest.apiKey().latestVersion(); if (header.apiVersion() == latestClientVersion) { log.trace("Sending {} {} with correlation id {} to node {}", clientRequest.apiKey(), request, - clientRequest.correlationId(), nodeId); + clientRequest.correlationId(), destination); } else { log.debug("Using older server API v{} to send {} {} with correlation id {} to node {}", - header.apiVersion(), clientRequest.apiKey(), request, clientRequest.correlationId(), nodeId); + header.apiVersion(), clientRequest.apiKey(), request, clientRequest.correlationId(), destination); } } - Send send = request.toSend(nodeId, header); + Send send = request.toSend(destination, header); InFlightRequest inFlightRequest = new InFlightRequest( + clientRequest, header, - clientRequest.createdTimeMs(), - clientRequest.destination(), - clientRequest.callback(), - clientRequest.expectResponse(), isInternalRequest, request, send, now); this.inFlightRequests.add(inFlightRequest); - selector.send(inFlightRequest.send); + selector.send(send); } /** @@ -464,6 +508,8 @@ private void doSend(ClientRequest clientRequest, boolean isInternalRequest, long */ @Override public List poll(long timeout, long now) { + ensureActive(); + if (!abortedSends.isEmpty()) { // If there are aborted sends because of unsupported version exceptions or disconnects, // handle them immediately without waiting for Selector#poll. @@ -475,7 +521,7 @@ public List poll(long timeout, long now) { long metadataTimeout = metadataUpdater.maybeUpdate(now); try { - this.selector.poll(Utils.min(timeout, metadataTimeout, requestTimeoutMs)); + this.selector.poll(Utils.min(timeout, metadataTimeout, defaultRequestTimeoutMs)); } catch (IOException e) { log.error("Unexpected error during I/O", e); } @@ -543,12 +589,35 @@ public void wakeup() { this.selector.wakeup(); } + @Override + public void initiateClose() { + if (state.compareAndSet(State.ACTIVE, State.CLOSING)) { + wakeup(); + } + } + + @Override + public boolean active() { + return state.get() == State.ACTIVE; + } + + private void ensureActive() { + if (!active()) + throw new DisconnectException("NetworkClient is no longer active, state is " + state); + } + /** * Close the network client */ @Override public void close() { - this.selector.close(); + state.compareAndSet(State.ACTIVE, State.CLOSING); + if (state.compareAndSet(State.CLOSING, State.CLOSED)) { + this.selector.close(); + this.metadataUpdater.close(); + } else { + log.warn("Attempting to close NetworkClient that has already been closed."); + } } /** @@ -658,7 +727,7 @@ else if (request.header.apiKey() == ApiKeys.METADATA) * @param now The current time */ private void handleTimedOutRequests(List responses, long now) { - List nodeIds = this.inFlightRequests.getNodesWithTimedOutRequests(now, this.requestTimeoutMs); + List nodeIds = this.inFlightRequests.nodesWithTimedOutRequests(now); for (String nodeId : nodeIds) { // close connection to the node this.selector.close(nodeId); @@ -837,7 +906,7 @@ private void initiateConnect(Node node, long now) { connectionStates.disconnected(nodeConnectionId, now); /* maybe the problem is our metadata, update it */ metadataUpdater.requestUpdate(); - log.debug("Error connecting to node {}", node, e); + log.warn("Error connecting to node {}", node, e); } } @@ -868,7 +937,7 @@ public boolean isUpdateDue(long now) { public long maybeUpdate(long now) { // should we update our metadata? long timeToNextMetadataUpdate = metadata.timeToNextUpdate(now); - long waitForMetadataFetch = this.metadataFetchInProgress ? requestTimeoutMs : 0; + long waitForMetadataFetch = this.metadataFetchInProgress ? defaultRequestTimeoutMs : 0; long metadataTimeout = Math.max(timeToNextMetadataUpdate, waitForMetadataFetch); @@ -915,6 +984,20 @@ public void handleAuthenticationFailure(AuthenticationException exception) { public void handleCompletedMetadataResponse(RequestHeader requestHeader, long now, MetadataResponse response) { this.metadataFetchInProgress = false; Cluster cluster = response.cluster(); + + // If any partition has leader with missing listeners, log a few for diagnosing broker configuration + // issues. This could be a transient issue if listeners were added dynamically to brokers. + List missingListenerPartitions = response.topicMetadata().stream().flatMap(topicMetadata -> + topicMetadata.partitionMetadata().stream() + .filter(partitionMetadata -> partitionMetadata.error() == Errors.LISTENER_NOT_FOUND) + .map(partitionMetadata -> new TopicPartition(topicMetadata.topic(), partitionMetadata.partition()))) + .collect(Collectors.toList()); + if (!missingListenerPartitions.isEmpty()) { + int count = missingListenerPartitions.size(); + log.warn("{} partitions have leader brokers without a matching listener, including {}", + count, missingListenerPartitions.subList(0, Math.min(10, count))); + } + // check if any topics metadata failed to get updated Map errors = response.errors(); if (!errors.isEmpty()) @@ -935,6 +1018,11 @@ public void requestUpdate() { this.metadata.requestUpdate(); } + @Override + public void close() { + this.metadata.close(); + } + /** * Return true if there's at least one connection establishment is currently underway */ @@ -965,7 +1053,7 @@ private long maybeUpdate(long now, Node node) { log.debug("Sending metadata request {} to node {}", metadataRequest, node); sendInternalMetadataRequest(metadataRequest, nodeConnectionId, now); - return requestTimeoutMs; + return defaultRequestTimeoutMs; } // If there's any connection establishment underway, wait until it completes. This prevents @@ -993,16 +1081,26 @@ private long maybeUpdate(long now, Node node) { } @Override - public ClientRequest newClientRequest(String nodeId, AbstractRequest.Builder requestBuilder, long createdTimeMs, + public ClientRequest newClientRequest(String nodeId, + AbstractRequest.Builder requestBuilder, + long createdTimeMs, boolean expectResponse) { - return newClientRequest(nodeId, requestBuilder, createdTimeMs, expectResponse, null); + return newClientRequest(nodeId, requestBuilder, createdTimeMs, expectResponse, defaultRequestTimeoutMs, null); } @Override - public ClientRequest newClientRequest(String nodeId, AbstractRequest.Builder requestBuilder, long createdTimeMs, - boolean expectResponse, RequestCompletionHandler callback) { + public ClientRequest newClientRequest(String nodeId, + AbstractRequest.Builder requestBuilder, + long createdTimeMs, + boolean expectResponse, + int requestTimeoutMs, + RequestCompletionHandler callback) { return new ClientRequest(nodeId, requestBuilder, correlation++, clientId, createdTimeMs, expectResponse, - callback); + requestTimeoutMs, callback); + } + + public boolean discoverBrokerVersions() { + return discoverBrokerVersions; } static class InFlightRequest { @@ -1015,8 +1113,28 @@ static class InFlightRequest { final Send send; final long sendTimeMs; final long createdTimeMs; + final long requestTimeoutMs; + + public InFlightRequest(ClientRequest clientRequest, + RequestHeader header, + boolean isInternalRequest, + AbstractRequest request, + Send send, + long sendTimeMs) { + this(header, + clientRequest.requestTimeoutMs(), + clientRequest.createdTimeMs(), + clientRequest.destination(), + clientRequest.callback(), + clientRequest.expectResponse(), + isInternalRequest, + request, + send, + sendTimeMs); + } public InFlightRequest(RequestHeader header, + int requestTimeoutMs, long createdTimeMs, String destination, RequestCompletionHandler callback, @@ -1026,6 +1144,8 @@ public InFlightRequest(RequestHeader header, Send send, long sendTimeMs) { this.header = header; + this.requestTimeoutMs = requestTimeoutMs; + this.createdTimeMs = createdTimeMs; this.destination = destination; this.callback = callback; this.expectResponse = expectResponse; @@ -1033,7 +1153,6 @@ public InFlightRequest(RequestHeader header, this.request = request; this.send = send; this.sendTimeMs = sendTimeMs; - this.createdTimeMs = createdTimeMs; } public ClientResponse completed(AbstractResponse response, long timeMs) { @@ -1060,7 +1179,4 @@ public String toString() { } } - public boolean discoverBrokerVersions() { - return discoverBrokerVersions; - } } diff --git a/clients/src/main/java/org/apache/kafka/clients/NetworkClientUtils.java b/clients/src/main/java/org/apache/kafka/clients/NetworkClientUtils.java index 94fe288090ec3..c952b82462def 100644 --- a/clients/src/main/java/org/apache/kafka/clients/NetworkClientUtils.java +++ b/clients/src/main/java/org/apache/kafka/clients/NetworkClientUtils.java @@ -18,6 +18,7 @@ package org.apache.kafka.clients; import org.apache.kafka.common.Node; +import org.apache.kafka.common.errors.DisconnectException; import org.apache.kafka.common.utils.Time; import java.io.IOException; @@ -83,25 +84,35 @@ public static boolean awaitReady(KafkaClient client, Node node, Time time, long * disconnection happens (which can happen for a number of reasons including a request timeout). * * In case of a disconnection, an `IOException` is thrown. + * If shutdown is initiated on the client during this method, an IOException is thrown. * * This method is useful for implementing blocking behaviour on top of the non-blocking `NetworkClient`, use it with * care. */ public static ClientResponse sendAndReceive(KafkaClient client, ClientRequest request, Time time) throws IOException { - client.send(request, time.milliseconds()); - while (true) { - List responses = client.poll(Long.MAX_VALUE, time.milliseconds()); - for (ClientResponse response : responses) { - if (response.requestHeader().correlationId() == request.correlationId()) { - if (response.wasDisconnected()) { - throw new IOException("Connection to " + response.destination() + " was disconnected before the response was read"); + try { + client.send(request, time.milliseconds()); + while (client.active()) { + List responses = client.poll(Long.MAX_VALUE, time.milliseconds()); + for (ClientResponse response : responses) { + if (response.requestHeader().correlationId() == request.correlationId()) { + if (response.wasDisconnected()) { + throw new IOException("Connection to " + response.destination() + " was disconnected before the response was read"); + } + if (response.versionMismatch() != null) { + throw response.versionMismatch(); + } + return response; } - if (response.versionMismatch() != null) { - throw response.versionMismatch(); - } - return response; } } + throw new IOException("Client was shutdown before response was read"); + } catch (DisconnectException e) { + if (client.active()) + throw e; + else + throw new IOException("Client was shutdown before response was read"); + } } } diff --git a/clients/src/main/java/org/apache/kafka/clients/admin/KafkaAdminClient.java b/clients/src/main/java/org/apache/kafka/clients/admin/KafkaAdminClient.java index 450de06fcd104..5383536390e4a 100644 --- a/clients/src/main/java/org/apache/kafka/clients/admin/KafkaAdminClient.java +++ b/clients/src/main/java/org/apache/kafka/clients/admin/KafkaAdminClient.java @@ -112,8 +112,6 @@ import org.apache.kafka.common.requests.OffsetFetchResponse; import org.apache.kafka.common.requests.RenewDelegationTokenRequest; import org.apache.kafka.common.requests.RenewDelegationTokenResponse; -import org.apache.kafka.common.requests.Resource; -import org.apache.kafka.common.requests.ResourceType; import org.apache.kafka.common.security.token.delegation.DelegationToken; import org.apache.kafka.common.security.token.delegation.TokenInformation; import org.apache.kafka.common.utils.AppInfoParser; @@ -166,6 +164,11 @@ public class KafkaAdminClient extends AdminClient { */ private static final long INVALID_SHUTDOWN_TIME = -1; + /** + * Thread name prefix for admin client network thread + */ + static final String NETWORK_THREAD_PREFIX = "kafka-admin-client-thread"; + private final Logger log; /** @@ -409,7 +412,7 @@ private KafkaAdminClient(AdminClientConfig config, this.metrics = metrics; this.client = client; this.runnable = new AdminClientRunnable(); - String threadName = "kafka-admin-client-thread | " + clientId; + String threadName = NETWORK_THREAD_PREFIX + " | " + clientId; this.thread = new KafkaThread(threadName, runnable, true); this.timeoutProcessorFactory = (timeoutProcessorFactory == null) ? new TimeoutProcessorFactory() : timeoutProcessorFactory; @@ -1683,19 +1686,19 @@ public DescribeConfigsResult describeConfigs(Collection configRe // The BROKER resources which we want to describe. We must make a separate DescribeConfigs // request for every BROKER resource we want to describe. - final Collection brokerResources = new ArrayList<>(); + final Collection brokerResources = new ArrayList<>(); // The non-BROKER resources which we want to describe. These resources can be described by a // single, unified DescribeConfigs request. - final Collection unifiedRequestResources = new ArrayList<>(configResources.size()); + final Collection unifiedRequestResources = new ArrayList<>(configResources.size()); for (ConfigResource resource : configResources) { if (resource.type() == ConfigResource.Type.BROKER && !resource.isDefault()) { - brokerFutures.put(resource, new KafkaFutureImpl()); - brokerResources.add(configResourceToResource(resource)); + brokerFutures.put(resource, new KafkaFutureImpl<>()); + brokerResources.add(resource); } else { - unifiedRequestFutures.put(resource, new KafkaFutureImpl()); - unifiedRequestResources.add(configResourceToResource(resource)); + unifiedRequestFutures.put(resource, new KafkaFutureImpl<>()); + unifiedRequestResources.add(resource); } } @@ -1716,7 +1719,7 @@ void handleResponse(AbstractResponse abstractResponse) { for (Map.Entry> entry : unifiedRequestFutures.entrySet()) { ConfigResource configResource = entry.getKey(); KafkaFutureImpl future = entry.getValue(); - DescribeConfigsResponse.Config config = response.config(configResourceToResource(configResource)); + DescribeConfigsResponse.Config config = response.config(configResource); if (config == null) { future.completeExceptionally(new UnknownServerException( "Malformed broker response: missing config for " + configResource)); @@ -1746,7 +1749,7 @@ void handleFailure(Throwable throwable) { for (Map.Entry> entry : brokerFutures.entrySet()) { final KafkaFutureImpl brokerFuture = entry.getValue(); - final Resource resource = configResourceToResource(entry.getKey()); + final ConfigResource resource = entry.getKey(); final int nodeId = Integer.parseInt(resource.name()); runnable.call(new Call("describeBrokerConfigs", calcDeadlineMs(now, options.timeoutMs()), new ConstantNodeIdProvider(nodeId)) { @@ -1792,21 +1795,6 @@ void handleFailure(Throwable throwable) { return new DescribeConfigsResult(allFutures); } - private Resource configResourceToResource(ConfigResource configResource) { - ResourceType resourceType; - switch (configResource.type()) { - case TOPIC: - resourceType = ResourceType.TOPIC; - break; - case BROKER: - resourceType = ResourceType.BROKER; - break; - default: - throw new IllegalArgumentException("Unexpected resource type " + configResource.type()); - } - return new Resource(resourceType, configResource.name()); - } - private List configSynonyms(DescribeConfigsResponse.ConfigEntry configEntry) { List synonyms = new ArrayList<>(configEntry.synonyms().size()); for (DescribeConfigsResponse.ConfigSynonym synonym : configEntry.synonyms()) { @@ -1856,7 +1844,7 @@ public AlterConfigsResult alterConfigs(Map configs, fina } if (!unifiedRequestResources.isEmpty()) allFutures.putAll(alterConfigs(configs, options, unifiedRequestResources, new LeastLoadedNodeProvider())); - return new AlterConfigsResult(new HashMap>(allFutures)); + return new AlterConfigsResult(new HashMap<>(allFutures)); } private Map> alterConfigs(Map configs, @@ -1864,13 +1852,13 @@ private Map> alterConfigs(Map resources, NodeProvider nodeProvider) { final Map> futures = new HashMap<>(); - final Map requestMap = new HashMap<>(resources.size()); + final Map requestMap = new HashMap<>(resources.size()); for (ConfigResource resource : resources) { List configEntries = new ArrayList<>(); for (ConfigEntry configEntry: configs.get(resource).entries()) configEntries.add(new AlterConfigsRequest.ConfigEntry(configEntry.name(), configEntry.value())); - requestMap.put(configResourceToResource(resource), new AlterConfigsRequest.Config(configEntries)); - futures.put(resource, new KafkaFutureImpl()); + requestMap.put(resource, new AlterConfigsRequest.Config(configEntries)); + futures.put(resource, new KafkaFutureImpl<>()); } final long now = time.milliseconds(); @@ -1886,7 +1874,7 @@ public void handleResponse(AbstractResponse abstractResponse) { AlterConfigsResponse response = (AlterConfigsResponse) abstractResponse; for (Map.Entry> entry : futures.entrySet()) { KafkaFutureImpl future = entry.getValue(); - ApiException exception = response.errors().get(configResourceToResource(entry.getKey())).exception(); + ApiException exception = response.errors().get(entry.getKey()).exception(); if (exception != null) { future.completeExceptionally(exception); } else { @@ -2407,16 +2395,9 @@ AbstractRequest.Builder createRequest(int timeoutMs) { @Override void handleResponse(AbstractResponse abstractResponse) { final FindCoordinatorResponse fcResponse = (FindCoordinatorResponse) abstractResponse; - Errors error = fcResponse.error(); - if (error == Errors.COORDINATOR_NOT_AVAILABLE) { - // Retry COORDINATOR_NOT_AVAILABLE, in case the error is temporary. - throw error.exception(); - } else if (error != Errors.NONE) { - // All other errors are immediate failures. - KafkaFutureImpl future = futures.get(groupId); - future.completeExceptionally(error.exception()); + + if (handleGroupRequestError(fcResponse.error(), futures.get(groupId))) return; - } final long nowDescribeConsumerGroups = time.milliseconds(); final int nodeId = fcResponse.node().id(); @@ -2433,39 +2414,36 @@ void handleResponse(AbstractResponse abstractResponse) { KafkaFutureImpl future = futures.get(groupId); final DescribeGroupsResponse.GroupMetadata groupMetadata = response.groups().get(groupId); - final Errors groupError = groupMetadata.error(); - if (groupError != Errors.NONE) { - // TODO: KAFKA-6789, we can retry based on the error code - future.completeExceptionally(groupError.exception()); - } else { - final String protocolType = groupMetadata.protocolType(); - if (protocolType.equals(ConsumerProtocol.PROTOCOL_TYPE) || protocolType.isEmpty()) { - final List members = groupMetadata.members(); - final List memberDescriptions = new ArrayList<>(members.size()); - - for (DescribeGroupsResponse.GroupMember groupMember : members) { - Set partitions = Collections.emptySet(); - if (groupMember.memberAssignment().remaining() > 0) { - final PartitionAssignor.Assignment assignment = ConsumerProtocol. - deserializeAssignment(groupMember.memberAssignment().duplicate()); - partitions = new HashSet<>(assignment.partitions()); - } - final MemberDescription memberDescription = - new MemberDescription(groupMember.memberId(), - groupMember.clientId(), - groupMember.clientHost(), - new MemberAssignment(partitions)); - memberDescriptions.add(memberDescription); + if (handleGroupRequestError(groupMetadata.error(), future)) + return; + + final String protocolType = groupMetadata.protocolType(); + if (protocolType.equals(ConsumerProtocol.PROTOCOL_TYPE) || protocolType.isEmpty()) { + final List members = groupMetadata.members(); + final List memberDescriptions = new ArrayList<>(members.size()); + + for (DescribeGroupsResponse.GroupMember groupMember : members) { + Set partitions = Collections.emptySet(); + if (groupMember.memberAssignment().remaining() > 0) { + final PartitionAssignor.Assignment assignment = ConsumerProtocol. + deserializeAssignment(groupMember.memberAssignment().duplicate()); + partitions = new HashSet<>(assignment.partitions()); } - final ConsumerGroupDescription consumerGroupDescription = - new ConsumerGroupDescription(groupId, - protocolType.isEmpty(), - memberDescriptions, - groupMetadata.protocol(), - ConsumerGroupState.parse(groupMetadata.state()), - fcResponse.node()); - future.complete(consumerGroupDescription); + final MemberDescription memberDescription = + new MemberDescription(groupMember.memberId(), + groupMember.clientId(), + groupMember.clientHost(), + new MemberAssignment(partitions)); + memberDescriptions.add(memberDescription); } + final ConsumerGroupDescription consumerGroupDescription = + new ConsumerGroupDescription(groupId, + protocolType.isEmpty(), + memberDescriptions, + groupMetadata.protocol(), + ConsumerGroupState.parse(groupMetadata.state()), + fcResponse.node()); + future.complete(consumerGroupDescription); } } @@ -2488,6 +2466,17 @@ void handleFailure(Throwable throwable) { return new DescribeConsumerGroupsResult(new HashMap>(futures)); } + + private boolean handleGroupRequestError(Errors error, KafkaFutureImpl future) { + if (error == Errors.COORDINATOR_LOAD_IN_PROGRESS || error == Errors.COORDINATOR_NOT_AVAILABLE) { + throw error.exception(); + } else if (error != Errors.NONE) { + future.completeExceptionally(error.exception()); + return true; + } + return false; + } + private final static class ListConsumerGroupsResults { private final List errors; private final HashMap listings; @@ -2573,8 +2562,11 @@ private void maybeAddConsumerGroup(ListGroupsResponse.Group group) { void handleResponse(AbstractResponse abstractResponse) { final ListGroupsResponse response = (ListGroupsResponse) abstractResponse; synchronized (results) { - if (response.error() != Errors.NONE) { - results.addError(response.error().exception(), node); + Errors error = response.error(); + if (error == Errors.COORDINATOR_LOAD_IN_PROGRESS || error == Errors.COORDINATOR_NOT_AVAILABLE) { + throw error.exception(); + } else if (error != Errors.NONE) { + results.addError(error.exception(), node); } else { for (ListGroupsResponse.Group group : response.groups()) { maybeAddConsumerGroup(group); @@ -2622,6 +2614,9 @@ AbstractRequest.Builder createRequest(int timeoutMs) { void handleResponse(AbstractResponse abstractResponse) { final FindCoordinatorResponse response = (FindCoordinatorResponse) abstractResponse; + if (handleGroupRequestError(response.error(), groupOffsetListingFuture)) + return; + final long nowListConsumerGroupOffsets = time.milliseconds(); final int nodeId = response.node().id(); @@ -2637,24 +2632,23 @@ void handleResponse(AbstractResponse abstractResponse) { final OffsetFetchResponse response = (OffsetFetchResponse) abstractResponse; final Map groupOffsetsListing = new HashMap<>(); - if (response.hasError()) { - groupOffsetListingFuture.completeExceptionally(response.error().exception()); - } else { - for (Map.Entry entry : - response.responseData().entrySet()) { - final TopicPartition topicPartition = entry.getKey(); - final Errors error = entry.getValue().error; - - if (error == Errors.NONE) { - final Long offset = entry.getValue().offset; - final String metadata = entry.getValue().metadata; - groupOffsetsListing.put(topicPartition, new OffsetAndMetadata(offset, metadata)); - } else { - log.warn("Skipping return offset for {} due to error {}.", topicPartition, error); - } + if (handleGroupRequestError(response.error(), groupOffsetListingFuture)) + return; + + for (Map.Entry entry : + response.responseData().entrySet()) { + final TopicPartition topicPartition = entry.getKey(); + final Errors error = entry.getValue().error; + + if (error == Errors.NONE) { + final Long offset = entry.getValue().offset; + final String metadata = entry.getValue().metadata; + groupOffsetsListing.put(topicPartition, new OffsetAndMetadata(offset, metadata)); + } else { + log.warn("Skipping return offset for {} due to error {}.", topicPartition, error); } - groupOffsetListingFuture.complete(groupOffsetsListing); } + groupOffsetListingFuture.complete(groupOffsetsListing); } @Override @@ -2708,6 +2702,9 @@ AbstractRequest.Builder createRequest(int timeoutMs) { void handleResponse(AbstractResponse abstractResponse) { final FindCoordinatorResponse response = (FindCoordinatorResponse) abstractResponse; + if (handleGroupRequestError(response.error(), futures.get(groupId))) + return; + final long nowDeleteConsumerGroups = time.milliseconds(); final int nodeId = response.node().id(); @@ -2726,11 +2723,10 @@ void handleResponse(AbstractResponse abstractResponse) { KafkaFutureImpl future = futures.get(groupId); final Errors groupError = response.get(groupId); - if (groupError != Errors.NONE) { - future.completeExceptionally(groupError.exception()); - } else { - future.complete(null); - } + if (handleGroupRequestError(groupError, future)) + return; + + future.complete(null); } @Override diff --git a/clients/src/main/java/org/apache/kafka/clients/admin/internals/AdminMetadataManager.java b/clients/src/main/java/org/apache/kafka/clients/admin/internals/AdminMetadataManager.java index 85d3c28e8df51..1ad3991ca24d2 100644 --- a/clients/src/main/java/org/apache/kafka/clients/admin/internals/AdminMetadataManager.java +++ b/clients/src/main/java/org/apache/kafka/clients/admin/internals/AdminMetadataManager.java @@ -118,6 +118,10 @@ public void handleCompletedMetadataResponse(RequestHeader requestHeader, long no public void requestUpdate() { AdminMetadataManager.this.requestUpdate(); } + + @Override + public void close() { + } } /** 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 72e496cbd4697..fefeae343e00e 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 @@ -218,6 +218,10 @@ public class ConsumerConfig extends AbstractConfig { 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; + /** default.api.timeout.ms */ + public static final String DEFAULT_API_TIMEOUT_MS_CONFIG = "default.api.timeout.ms"; + public static final String DEFAULT_API_TIMEOUT_MS_DOC = "Specifies the timeout (in milliseconds) for consumer APIs that could block. This configuration is used as the default timeout for all consumer operations that do not explicitly accept a timeout parameter."; + /** interceptor.classes */ public static final String INTERCEPTOR_CLASSES_CONFIG = "interceptor.classes"; public static final String INTERCEPTOR_CLASSES_DOC = "A list of classes to use as interceptors. " @@ -399,10 +403,16 @@ public class ConsumerConfig extends AbstractConfig { VALUE_DESERIALIZER_CLASS_DOC) .define(REQUEST_TIMEOUT_MS_CONFIG, Type.INT, - 305000, // chosen to be higher than the default of max.poll.interval.ms + 30000, atLeast(0), Importance.MEDIUM, REQUEST_TIMEOUT_MS_DOC) + .define(DEFAULT_API_TIMEOUT_MS_CONFIG, + Type.INT, + 60 * 1000, + atLeast(0), + Importance.MEDIUM, + DEFAULT_API_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, 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 5bd6b935b3972..0818c9469c68c 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 @@ -25,6 +25,7 @@ import org.apache.kafka.clients.consumer.internals.ConsumerMetrics; import org.apache.kafka.clients.consumer.internals.ConsumerNetworkClient; import org.apache.kafka.clients.consumer.internals.Fetcher; +import org.apache.kafka.clients.consumer.internals.Heartbeat; import org.apache.kafka.clients.consumer.internals.NoOpConsumerRebalanceListener; import org.apache.kafka.clients.consumer.internals.PartitionAssignor; import org.apache.kafka.clients.consumer.internals.SubscriptionState; @@ -34,7 +35,6 @@ import org.apache.kafka.common.MetricName; import org.apache.kafka.common.PartitionInfo; import org.apache.kafka.common.TopicPartition; -import org.apache.kafka.common.config.ConfigException; import org.apache.kafka.common.errors.InterruptException; import org.apache.kafka.common.errors.TimeoutException; import org.apache.kafka.common.internals.ClusterResourceListeners; @@ -567,6 +567,7 @@ public class KafkaConsumer implements Consumer { private final Metadata metadata; private final long retryBackoffMs; private final long requestTimeoutMs; + private final int defaultApiTimeoutMs; private volatile boolean closed = false; private List assignors; @@ -666,10 +667,7 @@ private KafkaConsumer(ConsumerConfig config, log.debug("Initializing 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.defaultApiTimeoutMs = config.getInt(ConsumerConfig.DEFAULT_API_TIMEOUT_MS_CONFIG); this.time = Time.SYSTEM; Map metricsTags = Collections.singletonMap("client-id", clientId); @@ -748,12 +746,15 @@ private KafkaConsumer(ConsumerConfig config, this.assignors = config.getConfiguredInstances( ConsumerConfig.PARTITION_ASSIGNMENT_STRATEGY_CONFIG, PartitionAssignor.class); + + int maxPollIntervalMs = config.getInt(ConsumerConfig.MAX_POLL_INTERVAL_MS_CONFIG); + int sessionTimeoutMs = config.getInt(ConsumerConfig.SESSION_TIMEOUT_MS_CONFIG); this.coordinator = new ConsumerCoordinator(logContext, this.client, groupId, - config.getInt(ConsumerConfig.MAX_POLL_INTERVAL_MS_CONFIG), - config.getInt(ConsumerConfig.SESSION_TIMEOUT_MS_CONFIG), - heartbeatIntervalMs, + maxPollIntervalMs, + sessionTimeoutMs, + new Heartbeat(sessionTimeoutMs, heartbeatIntervalMs, maxPollIntervalMs, retryBackoffMs), assignors, this.metadata, this.subscriptions, @@ -814,6 +815,7 @@ private KafkaConsumer(ConsumerConfig config, Metadata metadata, long retryBackoffMs, long requestTimeoutMs, + int defaultApiTimeoutMs, List assignors) { this.log = logContext.logger(getClass()); this.clientId = clientId; @@ -829,6 +831,7 @@ private KafkaConsumer(ConsumerConfig config, this.metadata = metadata; this.retryBackoffMs = retryBackoffMs; this.requestTimeoutMs = requestTimeoutMs; + this.defaultApiTimeoutMs = defaultApiTimeoutMs; this.assignors = assignors; } @@ -1268,8 +1271,9 @@ private long remainingTimeAtLeastZero(final long timeoutMs, final long elapsedTi * 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 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). + * This is a synchronous commit and will block until either the commit succeeds, an unrecoverable error is + * encountered (in which case it is thrown to the caller), or the timeout specified by {@code default.api.timeout.ms} expires + * (in which case a {@link org.apache.kafka.common.errors.TimeoutException} is thrown to the caller). *

* Note that asynchronous offset commits sent previously with the {@link #commitAsync(OffsetCommitCallback)} * (or similar) are guaranteed to have their callbacks invoked prior to completion of this method. @@ -1286,10 +1290,12 @@ private long remainingTimeAtLeastZero(final long timeoutMs, final long elapsedTi * configured groupId. See the exception for more details * @throws org.apache.kafka.common.KafkaException for any other unrecoverable errors (e.g. if offset metadata * is too large or if the topic does not exist). + * @throws org.apache.kafka.common.errors.TimeoutException if the timeout specified by {@code default.api.timeout.ms} expires + * before successful completion of the offset commit */ @Override public void commitSync() { - commitSync(Duration.ofMillis(Long.MAX_VALUE)); + commitSync(Duration.ofMillis(defaultApiTimeoutMs)); } /** @@ -1343,7 +1349,8 @@ public void commitSync(Duration timeout) { * i.e. lastProcessedMessageOffset + 1. *

* 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). + * encountered (in which case it is thrown to the caller), or the timeout specified by {@code default.api.timeout.ms} expires + * (in which case a {@link org.apache.kafka.common.errors.TimeoutException} is thrown to the caller). *

* Note that asynchronous offset commits sent previously with the {@link #commitAsync(OffsetCommitCallback)} * (or similar) are guaranteed to have their callbacks invoked prior to completion of this method. @@ -1362,10 +1369,12 @@ public void commitSync(Duration timeout) { * @throws java.lang.IllegalArgumentException if the committed offset is negative * @throws org.apache.kafka.common.KafkaException for any other unrecoverable errors (e.g. if offset metadata * is too large or if the topic does not exist). + * @throws org.apache.kafka.common.errors.TimeoutException if the timeout expires before successful completion + * of the offset commit */ @Override public void commitSync(final Map offsets) { - commitSync(offsets, Duration.ofMillis(Long.MAX_VALUE)); + commitSync(offsets, Duration.ofMillis(defaultApiTimeoutMs)); } /** @@ -1560,7 +1569,8 @@ public void seekToEnd(Collection partitions) { * This method may issue a remote call to the server if there is no current position for the given partition. *

* This call will block until either the position could be determined or an unrecoverable error is - * encountered (in which case it is thrown to the caller). + * encountered (in which case it is thrown to the caller), or the timeout specified by {@code default.api.timeout.ms} expires + * (in which case a {@link org.apache.kafka.common.errors.TimeoutException} is thrown to the caller). * * @param partition The partition to get the position for * @return The current position of the consumer (that is, the offset of the next record to be fetched) @@ -1575,10 +1585,12 @@ public void seekToEnd(Collection partitions) { * @throws org.apache.kafka.common.errors.AuthorizationException if not authorized to the topic or to the * configured groupId. See the exception for more details * @throws org.apache.kafka.common.KafkaException for any other unrecoverable errors + * @throws org.apache.kafka.common.errors.TimeoutException if the position cannot be determined before the + * timeout specified by {@code default.api.timeout.ms} expires */ @Override public long position(TopicPartition partition) { - return position(partition, Duration.ofMillis(Long.MAX_VALUE)); + return position(partition, Duration.ofMillis(defaultApiTimeoutMs)); } /** @@ -1592,7 +1604,7 @@ public long position(TopicPartition partition) { * @param partition The partition to get the position for * @param timeout The maximum amount of time to await determination of the current position * @return The current position of the consumer (that is, the offset of the next record to be fetched) - * @throws IllegalArgumentException if the provided TopicPartition is not assigned to this consumer + * @throws IllegalStateException if the provided TopicPartition is not assigned to this consumer * @throws org.apache.kafka.clients.consumer.InvalidOffsetException if no offset is currently defined for * the partition * @throws org.apache.kafka.common.errors.WakeupException if {@link #wakeup()} is called before or while this @@ -1641,7 +1653,10 @@ public long position(TopicPartition partition, final Duration timeout) { * Get 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 will block to do a remote call to get the latest committed offsets from the server. + * This call will do a remote call to get the latest committed offset from the server, and will block until the + * committed offset is gotten successfully, an unrecoverable error is encountered (in which case it is thrown to + * the caller), or the timeout specified by {@code default.api.timeout.ms} expires (in which case a + * {@link org.apache.kafka.common.errors.TimeoutException} is thrown to the caller). * * @param partition The partition to check * @return The last committed offset and metadata or null if there was no prior commit @@ -1653,10 +1668,12 @@ public long position(TopicPartition partition, final Duration timeout) { * @throws org.apache.kafka.common.errors.AuthorizationException if not authorized to the topic or to the * configured groupId. See the exception for more details * @throws org.apache.kafka.common.KafkaException for any other unrecoverable errors + * @throws org.apache.kafka.common.errors.TimeoutException if the committed offset cannot be found before + * the timeout specified by {@code default.api.timeout.ms} expires. */ @Override public OffsetAndMetadata committed(TopicPartition partition) { - return committed(partition, Duration.ofMillis(Long.MAX_VALUE)); + return committed(partition, Duration.ofMillis(defaultApiTimeoutMs)); } /** @@ -1718,11 +1735,11 @@ public OffsetAndMetadata committed(TopicPartition partition, final Duration time * @throws org.apache.kafka.common.errors.AuthorizationException if not authorized to the specified topic. See the exception for more details * @throws org.apache.kafka.common.KafkaException for any other unrecoverable errors * @throws org.apache.kafka.common.errors.TimeoutException if the offset metadata could not be fetched before - * the amount of time allocated by {@code request.timeout.ms} expires. + * the amount of time allocated by {@code default.api.timeout.ms} expires. */ @Override public List partitionsFor(String topic) { - return partitionsFor(topic, Duration.ofMillis(requestTimeoutMs)); + return partitionsFor(topic, Duration.ofMillis(defaultApiTimeoutMs)); } /** @@ -1774,11 +1791,11 @@ public List partitionsFor(String topic, Duration timeout) { * this function is called * @throws org.apache.kafka.common.KafkaException for any other unrecoverable errors * @throws org.apache.kafka.common.errors.TimeoutException if the offset metadata could not be fetched before - * the amount of time allocated by {@code request.timeout.ms} expires. + * the amount of time allocated by {@code default.api.timeout.ms} expires. */ @Override public Map> listTopics() { - return listTopics(Duration.ofMillis(requestTimeoutMs)); + return listTopics(Duration.ofMillis(defaultApiTimeoutMs)); } /** @@ -1879,13 +1896,13 @@ public Set paused() { * @throws org.apache.kafka.common.errors.AuthorizationException if not authorized to the topic(s). See the exception for more details * @throws IllegalArgumentException if the target timestamp is negative * @throws org.apache.kafka.common.errors.TimeoutException if the offset metadata could not be fetched before - * the amount of time allocated by {@code request.timeout.ms} expires. + * the amount of time allocated by {@code default.api.timeout.ms} expires. * @throws org.apache.kafka.common.errors.UnsupportedVersionException if the broker does not support looking up * the offsets by timestamp */ @Override public Map offsetsForTimes(Map timestampsToSearch) { - return offsetsForTimes(timestampsToSearch, Duration.ofMillis(requestTimeoutMs)); + return offsetsForTimes(timestampsToSearch, Duration.ofMillis(defaultApiTimeoutMs)); } /** @@ -1939,11 +1956,11 @@ public Map offsetsForTimes(Map beginningOffsets(Collection partitions) { - return beginningOffsets(partitions, Duration.ofMillis(requestTimeoutMs)); + return beginningOffsets(partitions, Duration.ofMillis(defaultApiTimeoutMs)); } /** @@ -2062,7 +2079,7 @@ public void close() { @Deprecated @Override public void close(long timeout, TimeUnit timeUnit) { - close(Duration.ofMillis(TimeUnit.MILLISECONDS.toMillis(timeout))); + close(Duration.ofMillis(timeUnit.toMillis(timeout))); } /** @@ -2125,12 +2142,12 @@ private void close(long timeoutMs, boolean swallowException) { firstException.compareAndSet(null, t); log.error("Failed to close coordinator", t); } - ClientUtils.closeQuietly(fetcher, "fetcher", firstException); - ClientUtils.closeQuietly(interceptors, "consumer interceptors", 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); + Utils.closeQuietly(fetcher, "fetcher", firstException); + Utils.closeQuietly(interceptors, "consumer interceptors", firstException); + Utils.closeQuietly(metrics, "consumer metrics", firstException); + Utils.closeQuietly(client, "consumer network client", firstException); + Utils.closeQuietly(keyDeserializer, "consumer key deserializer", firstException); + Utils.closeQuietly(valueDeserializer, "consumer value deserializer", firstException); AppInfoParser.unregisterAppInfo(JMX_PREFIX, clientId, metrics); log.debug("Kafka consumer has been closed"); Throwable exception = firstException.get(); 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 cf1b07fabe21b..7f42d60bfe813 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 @@ -58,7 +58,8 @@ public class MockConsumer implements Consumer { private final Set paused; private Map>> records; - private KafkaException exception; + private KafkaException pollException; + private KafkaException offsetsException; private AtomicBoolean wakeup; private boolean closed; @@ -71,7 +72,7 @@ public MockConsumer(OffsetResetStrategy offsetResetStrategy) { this.beginningOffsets = new HashMap<>(); this.endOffsets = new HashMap<>(); this.pollTasks = new LinkedList<>(); - this.exception = null; + this.pollException = null; this.wakeup = new AtomicBoolean(false); this.committed = new HashMap<>(); } @@ -170,9 +171,9 @@ public synchronized ConsumerRecords poll(final Duration timeout) { throw new WakeupException(); } - if (exception != null) { - RuntimeException exception = this.exception; - this.exception = null; + if (pollException != null) { + RuntimeException exception = this.pollException; + this.pollException = null; throw exception; } @@ -183,16 +184,17 @@ public synchronized ConsumerRecords poll(final Duration timeout) { // update the consumed offset final Map>> results = new HashMap<>(); - for (final TopicPartition topicPartition : records.keySet()) { - results.put(topicPartition, new ArrayList>()); - } for (Map.Entry>> entry : this.records.entrySet()) { if (!subscriptions.isPaused(entry.getKey())) { final List> recs = entry.getValue(); for (final ConsumerRecord rec : recs) { + if (beginningOffsets.get(entry.getKey()) != null && beginningOffsets.get(entry.getKey()) > subscriptions.position(entry.getKey())) { + throw new OffsetOutOfRangeException(Collections.singletonMap(entry.getKey(), subscriptions.position(entry.getKey()))); + } + if (assignment().contains(entry.getKey()) && rec.offset() >= subscriptions.position(entry.getKey())) { - results.get(entry.getKey()).add(rec); + results.computeIfAbsent(entry.getKey(), partition -> new ArrayList<>()).add(rec); subscriptions.position(entry.getKey(), rec.offset() + 1); } } @@ -216,8 +218,20 @@ public synchronized void addRecord(ConsumerRecord record) { recs.add(record); } + /** + * @deprecated Use {@link #setPollException(KafkaException)} instead + */ + @Deprecated public synchronized void setException(KafkaException exception) { - this.exception = exception; + setPollException(exception); + } + + public synchronized void setPollException(KafkaException exception) { + this.pollException = exception; + } + + public synchronized void setOffsetsException(KafkaException exception) { + this.offsetsException = exception; } @Override @@ -385,6 +399,11 @@ public synchronized Map offsetsForTimes(Map< @Override public synchronized Map beginningOffsets(Collection partitions) { + if (offsetsException != null) { + RuntimeException exception = this.offsetsException; + this.offsetsException = null; + throw exception; + } Map result = new HashMap<>(); for (TopicPartition tp : partitions) { Long beginningOffset = beginningOffsets.get(tp); @@ -397,6 +416,11 @@ public synchronized Map beginningOffsets(Collection endOffsets(Collection partitions) { + if (offsetsException != null) { + RuntimeException exception = this.offsetsException; + this.offsetsException = null; + throw exception; + } Map result = new HashMap<>(); for (TopicPartition tp : partitions) { Long endOffset = getEndOffset(endOffsets.get(tp)); diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/RoundRobinAssignor.java b/clients/src/main/java/org/apache/kafka/clients/consumer/RoundRobinAssignor.java index 7e8d6f2c8077e..3b543f7130832 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/RoundRobinAssignor.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/RoundRobinAssignor.java @@ -48,7 +48,7 @@ * with 1, 2, and 3 partitions, respectively. Therefore, the partitions are t0p0, t1p0, t1p1, t2p0, * t2p1, t2p2. C0 is subscribed to t0; C1 is subscribed to t0, t1; and C2 is subscribed to t0, t1, t2. * - * Tha assignment will be: + * That assignment will be: * C0: [t0p0] * C1: [t1p0] * C2: [t1p1, t2p0, t2p1, t2p2] diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/AbstractCoordinator.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/AbstractCoordinator.java index adbaae776ab39..53834fb81dfba 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/AbstractCoordinator.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/AbstractCoordinator.java @@ -129,7 +129,7 @@ public AbstractCoordinator(LogContext logContext, String groupId, int rebalanceTimeoutMs, int sessionTimeoutMs, - int heartbeatIntervalMs, + Heartbeat heartbeat, Metrics metrics, String metricGrpPrefix, Time time, @@ -142,11 +142,27 @@ public AbstractCoordinator(LogContext logContext, this.rebalanceTimeoutMs = rebalanceTimeoutMs; this.sessionTimeoutMs = sessionTimeoutMs; this.leaveGroupOnClose = leaveGroupOnClose; - this.heartbeat = new Heartbeat(sessionTimeoutMs, heartbeatIntervalMs, rebalanceTimeoutMs, retryBackoffMs); + this.heartbeat = heartbeat; this.sensors = new GroupCoordinatorMetrics(metrics, metricGrpPrefix); this.retryBackoffMs = retryBackoffMs; } + public AbstractCoordinator(LogContext logContext, + ConsumerNetworkClient client, + String groupId, + int rebalanceTimeoutMs, + int sessionTimeoutMs, + int heartbeatIntervalMs, + Metrics metrics, + String metricGrpPrefix, + Time time, + long retryBackoffMs, + boolean leaveGroupOnClose) { + this(logContext, client, groupId, rebalanceTimeoutMs, sessionTimeoutMs, + new Heartbeat(sessionTimeoutMs, heartbeatIntervalMs, rebalanceTimeoutMs, retryBackoffMs), + metrics, metricGrpPrefix, time, retryBackoffMs, leaveGroupOnClose); + } + /** * Unique identifier for the class of supported protocols (e.g. "consumer" or "connect"). * @return Non-null protocol type name @@ -184,9 +200,8 @@ protected abstract Map performAssignment(String leaderId, Map allMemberMetadata); /** - * Invoked when a group member has successfully joined a group. If this call is woken up (i.e. - * if the invocation raises {@link org.apache.kafka.common.errors.WakeupException}), then it - * will be retried on the next call to {@link #ensureActiveGroup()}. + * Invoked when a group member has successfully joined a group. If this call fails with an exception, + * then it will be retried using the same assignment state on the next call to {@link #ensureActiveGroup()}. * * @param generation The generation that was joined * @param memberId The identifier for the local member in the group @@ -319,7 +334,11 @@ public void ensureActiveGroup() { * @return true iff the group is active */ boolean ensureActiveGroup(final long timeoutMs) { - final long startTime = time.milliseconds(); + return ensureActiveGroup(timeoutMs, time.milliseconds()); + } + + // Visible for testing + boolean ensureActiveGroup(long timeoutMs, long startMs) { // always ensure that the coordinator is ready because we may have been disconnected // when sending heartbeats and does not necessarily require us to rejoin the group. if (!ensureCoordinatorReady(timeoutMs)) { @@ -328,7 +347,9 @@ boolean ensureActiveGroup(final long timeoutMs) { startHeartbeatThreadIfNeeded(); - return joinGroupIfNeeded(remainingTimeAtLeastZero(timeoutMs, time.milliseconds() - startTime)); + long joinStartMs = time.milliseconds(); + long joinTimeoutMs = remainingTimeAtLeastZero(timeoutMs, joinStartMs - startMs); + return joinGroupIfNeeded(joinTimeoutMs, joinStartMs); } private synchronized void startHeartbeatThreadIfNeeded() { @@ -366,17 +387,17 @@ private void closeHeartbeatThread() { * Visible for testing. * * @param timeoutMs Time to complete this action + * @param startTimeMs Current time when invoked * @return true iff the operation succeeded */ - boolean joinGroupIfNeeded(final long timeoutMs) { - final long startTime = time.milliseconds(); + boolean joinGroupIfNeeded(final long timeoutMs, final long startTimeMs) { long elapsedTime = 0L; while (rejoinNeededOrPending()) { if (!ensureCoordinatorReady(remainingTimeAtLeastZero(timeoutMs, elapsedTime))) { return false; } - elapsedTime = time.milliseconds() - startTime; + elapsedTime = time.milliseconds() - startTimeMs; // call onJoinPrepare if needed. We set a flag to make sure that we do not call it a second // time if the client is woken up before a pending rebalance completes. This must be called @@ -396,7 +417,9 @@ boolean joinGroupIfNeeded(final long timeoutMs) { } if (future.succeeded()) { - onJoinComplete(generation.generationId, generation.memberId, generation.protocol, future.value()); + // Duplicate the buffer in case `onJoinComplete` does not complete and needs to be retried. + ByteBuffer memberAssignment = future.value().duplicate(); + onJoinComplete(generation.generationId, generation.memberId, generation.protocol, memberAssignment); // We reset the join group future only after the completion callback returns. This ensures // that if the callback is woken up, we will retry it on the next joinGroupIfNeeded. @@ -415,7 +438,7 @@ else if (!future.isRetriable()) } if (rejoinNeededOrPending()) { - elapsedTime = time.milliseconds() - startTime; + elapsedTime = time.milliseconds() - startTimeMs; } } return true; @@ -473,9 +496,12 @@ public void onFailure(RuntimeException e) { * Join the group and return the assignment for the next generation. This function handles both * JoinGroup and SyncGroup, delegating to {@link #performAssignment(String, String, Map)} if * elected leader by the coordinator. + * + * NOTE: This is visible only for testing + * * @return A request future which wraps the assignment returned from the group leader */ - private RequestFuture sendJoinGroupRequest() { + RequestFuture sendJoinGroupRequest() { if (coordinatorUnknown()) return RequestFuture.coordinatorNotAvailable(); @@ -489,7 +515,12 @@ private RequestFuture sendJoinGroupRequest() { metadata()).setRebalanceTimeout(this.rebalanceTimeoutMs); log.debug("Sending JoinGroup ({}) to coordinator {}", requestBuilder, this.coordinator); - return client.send(coordinator, requestBuilder) + + // Note that we override the request timeout using the rebalance timeout since that is the + // maximum time that it may block on the coordinator. We add an extra 5 seconds for small delays. + + int joinGroupTimeoutMs = Math.max(rebalanceTimeoutMs, rebalanceTimeoutMs + 5000); + return client.send(coordinator, requestBuilder, joinGroupTimeoutMs) .compose(new JoinGroupResponseHandler()); } diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ConsumerCoordinator.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ConsumerCoordinator.java index 9c19af1703779..f078f8956aef0 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ConsumerCoordinator.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ConsumerCoordinator.java @@ -118,7 +118,7 @@ public ConsumerCoordinator(LogContext logContext, String groupId, int rebalanceTimeoutMs, int sessionTimeoutMs, - int heartbeatIntervalMs, + Heartbeat heartbeat, List assignors, Metadata metadata, SubscriptionState subscriptions, @@ -136,7 +136,7 @@ public ConsumerCoordinator(LogContext logContext, groupId, rebalanceTimeoutMs, sessionTimeoutMs, - heartbeatIntervalMs, + heartbeat, metrics, metricGrpPrefix, time, @@ -264,10 +264,10 @@ protected void onJoinComplete(int generation, this.joinedSubscription = newJoinedSubscription; } - // update the metadata and enforce a refresh to make sure the fetcher can start - // fetching data in the next iteration + // Update the metadata to include the full group subscription. The leader will trigger a rebalance + // if there are any metadata changes affecting any of the consumed partitions (whether or not this + // instance is subscribed to the topics). this.metadata.setTopics(subscriptions.groupSubscription()); - if (!client.ensureFreshMetadata(Long.MAX_VALUE)) throw new TimeoutException(); // give the assignor a chance to update internal state based on the received assignment assignor.onAssignment(assignment); @@ -306,13 +306,16 @@ public boolean poll(final long timeoutMs) { invokeCompletedOffsetCommitCallbacks(); if (subscriptions.partitionsAutoAssigned()) { + // Always update the heartbeat last poll time so that the heartbeat thread does not leave the + // group proactively due to application inactivity even if (say) the coordinator cannot be found. + pollHeartbeat(currentTime); + if (coordinatorUnknown()) { if (!ensureCoordinatorReady(remainingTimeAtLeastZero(timeoutMs, elapsed))) { return false; } currentTime = time.milliseconds(); elapsed = currentTime - startTime; - } if (rejoinNeededOrPending()) { @@ -320,6 +323,16 @@ public boolean poll(final long timeoutMs) { // we need to ensure that the metadata is fresh before joining initially. This ensures // that we have matched the pattern against the cluster's topics at least once before joining. if (subscriptions.hasPatternSubscription()) { + // For consumer group that uses pattern-based subscription, after a topic is created, + // any consumer that discovers the topic after metadata refresh can trigger rebalance + // across the entire consumer group. Multiple rebalances can be triggered after one topic + // creation if consumers refresh metadata at vastly different times. We can significantly + // reduce the number of rebalances caused by single topic creation by asking consumer to + // refresh metadata before re-joining the group as long as the refresh backoff time has + // passed. + if (this.metadata.timeToAllowUpdate(currentTime) == 0) { + this.metadata.requestUpdate(); + } if (!client.ensureFreshMetadata(remainingTimeAtLeastZero(timeoutMs, elapsed))) { return false; } @@ -333,8 +346,6 @@ public boolean poll(final long timeoutMs) { currentTime = time.milliseconds(); } - - pollHeartbeat(currentTime); } else { // For manually assigned partitions, if there are no ready nodes, await metadata. // If connections to all nodes fail, wakeups triggered while attempting to send fetch @@ -829,7 +840,11 @@ public void handle(OffsetCommitResponse commitResponse, RequestFuture futu if (error == Errors.NONE) { log.debug("Committed offset {} for partition {}", offset, tp); } else { - log.error("Offset commit failed on partition {} at offset {}: {}", tp, offset, error.message()); + if (error.exception() instanceof RetriableException) { + log.warn("Offset commit failed on partition {} at offset {}: {}", tp, offset, error.message()); + } else { + log.error("Offset commit failed on partition {} at offset {}: {}", tp, offset, error.message()); + } if (error == Errors.GROUP_AUTHORIZATION_FAILED) { future.raise(new GroupAuthorizationException(groupId)); @@ -964,7 +979,8 @@ private ConsumerCoordinatorMetrics(Metrics metrics, String metricGrpPrefix) { Measurable numParts = new Measurable() { public double measure(MetricConfig config, long now) { - return subscriptions.assignedPartitions().size(); + // Get the number of assigned partitions in a thread safe manner + return subscriptions.numAssignedPartitions(); } }; metrics.addMetric(metrics.metricName("assigned-partitions", 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 index a9e167a9a2e7f..0bf0aad1b945c 100644 --- 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 @@ -62,7 +62,7 @@ public class ConsumerNetworkClient implements Closeable { private final Time time; private final long retryBackoffMs; private final int maxPollTimeoutMs; - private final long unsentExpiryMs; + private final int requestTimeoutMs; private final AtomicBoolean wakeupDisabled = new AtomicBoolean(); // We do not need high throughput, so use a fair lock to try to avoid starvation @@ -83,7 +83,7 @@ public ConsumerNetworkClient(LogContext logContext, Metadata metadata, Time time, long retryBackoffMs, - long requestTimeoutMs, + int requestTimeoutMs, int maxPollTimeoutMs) { this.log = logContext.logger(ConsumerNetworkClient.class); this.client = client; @@ -91,7 +91,15 @@ public ConsumerNetworkClient(LogContext logContext, this.time = time; this.retryBackoffMs = retryBackoffMs; this.maxPollTimeoutMs = Math.min(maxPollTimeoutMs, MAX_POLL_TIMEOUT_MS); - this.unsentExpiryMs = requestTimeoutMs; + this.requestTimeoutMs = requestTimeoutMs; + } + + + /** + * Send a request with the default timeout. See {@link #send(Node, AbstractRequest.Builder, int)}. + */ + public RequestFuture send(Node node, AbstractRequest.Builder requestBuilder) { + return send(node, requestBuilder, requestTimeoutMs); } /** @@ -104,13 +112,18 @@ public ConsumerNetworkClient(LogContext logContext, * * @param node The destination of the request * @param requestBuilder A builder for the request payload + * @param requestTimeoutMs Maximum time in milliseconds to await a response before disconnecting the socket and + * cancelling the request. The request may be cancelled sooner if the socket disconnects + * for any reason. * @return A future which indicates the result of the send. */ - public RequestFuture send(Node node, AbstractRequest.Builder requestBuilder) { + public RequestFuture send(Node node, + AbstractRequest.Builder requestBuilder, + int requestTimeoutMs) { long now = time.milliseconds(); RequestFutureCompletionHandler completionHandler = new RequestFutureCompletionHandler(); ClientRequest clientRequest = client.newClientRequest(node.idString(), requestBuilder, now, true, - completionHandler); + requestTimeoutMs, completionHandler); unsent.put(node, clientRequest); // wakeup the client in case it is blocking in poll so that we can send the queued request @@ -136,13 +149,6 @@ public boolean hasReadyNodes(long now) { } } - /** - * Block until the metadata has been refreshed. - */ - public void awaitMetadataUpdate() { - awaitMetadataUpdate(Long.MAX_VALUE); - } - /** * Block waiting on the metadata refresh with a timeout. * @@ -444,10 +450,10 @@ public void disconnectAsync(Node node) { private void failExpiredRequests(long now) { // clear all expired unsent requests and fail their corresponding futures - Collection expiredRequests = unsent.removeExpiredRequests(now, unsentExpiryMs); + Collection expiredRequests = unsent.removeExpiredRequests(now); for (ClientRequest request : expiredRequests) { RequestFutureCompletionHandler handler = (RequestFutureCompletionHandler) request.callback(); - handler.onFailure(new TimeoutException("Failed to send request after " + unsentExpiryMs + " ms.")); + handler.onFailure(new TimeoutException("Failed to send request after " + request.requestTimeoutMs() + " ms.")); } } @@ -655,13 +661,14 @@ public boolean hasRequests() { return false; } - public Collection removeExpiredRequests(long now, long unsentExpiryMs) { + private Collection removeExpiredRequests(long now) { List expiredRequests = new ArrayList<>(); for (ConcurrentLinkedQueue requests : unsent.values()) { Iterator requestIterator = requests.iterator(); while (requestIterator.hasNext()) { ClientRequest request = requestIterator.next(); - if (request.createdTimeMs() < now - unsentExpiryMs) { + long elapsedMs = Math.max(0, now - request.createdTimeMs()); + if (elapsedMs > request.requestTimeoutMs()) { expiredRequests.add(request); requestIterator.remove(); } else 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 index ca8e0d26c81e2..cf328735a16cb 100644 --- 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 @@ -91,7 +91,21 @@ import static org.apache.kafka.common.serialization.ExtendedDeserializer.Wrapper.ensureExtended; /** - * This class manage the fetching process with the brokers. + * This class manages the fetching process with the brokers. + *

+ * Thread-safety: + * Requests and responses of Fetcher may be processed by different threads since heartbeat + * thread may process responses. Other operations are single-threaded and invoked only from + * the thread polling the consumer. + *

    + *
  • If a response handler accesses any shared state of the Fetcher (e.g. FetchSessionHandler), + * all access to that state must be synchronized on the Fetcher instance.
  • + *
  • If a response handler accesses any shared state of the coordinator (e.g. SubscriptionState), + * it is assumed that all access to that state is synchronized on the coordinator instance by + * the caller.
  • + *
  • Responses that collate partial responses from multiple brokers (e.g. to list offsets) are + * synchronized on the response future.
  • + *
*/ public class Fetcher implements SubscriptionState.Listener, Closeable { private final Logger log; @@ -187,7 +201,7 @@ public boolean hasCompletedFetches() { * an in-flight fetch or pending fetch data. * @return number of fetches sent */ - public int sendFetches() { + public synchronized int sendFetches() { Map fetchRequestMap = prepareFetchRequests(); for (Map.Entry entry : fetchRequestMap.entrySet()) { final Node fetchTarget = entry.getKey(); @@ -205,39 +219,43 @@ public int sendFetches() { .addListener(new RequestFutureListener() { @Override public void onSuccess(ClientResponse resp) { - FetchResponse response = (FetchResponse) resp.responseBody(); - FetchSessionHandler handler = sessionHandlers.get(fetchTarget.id()); - if (handler == null) { - log.error("Unable to find FetchSessionHandler for node {}. Ignoring fetch response.", - fetchTarget.id()); - return; + synchronized (Fetcher.this) { + FetchResponse response = (FetchResponse) resp.responseBody(); + FetchSessionHandler handler = sessionHandler(fetchTarget.id()); + if (handler == null) { + log.error("Unable to find FetchSessionHandler for node {}. Ignoring fetch response.", + fetchTarget.id()); + return; + } + if (!handler.handleResponse(response)) { + return; + } + + Set partitions = new HashSet<>(response.responseData().keySet()); + FetchResponseMetricAggregator metricAggregator = new FetchResponseMetricAggregator(sensors, partitions); + + for (Map.Entry> entry : response.responseData().entrySet()) { + TopicPartition partition = entry.getKey(); + long fetchOffset = data.sessionPartitions().get(partition).fetchOffset; + FetchResponse.PartitionData fetchData = entry.getValue(); + + log.debug("Fetch {} at offset {} for partition {} returned fetch data {}", + isolationLevel, fetchOffset, partition, fetchData); + completedFetches.add(new CompletedFetch(partition, fetchOffset, fetchData, metricAggregator, + resp.requestHeader().apiVersion())); + } + + sensors.fetchLatency.record(resp.requestLatencyMs()); } - if (!handler.handleResponse(response)) { - return; - } - - Set partitions = new HashSet<>(response.responseData().keySet()); - FetchResponseMetricAggregator metricAggregator = new FetchResponseMetricAggregator(sensors, partitions); - - for (Map.Entry> entry : response.responseData().entrySet()) { - TopicPartition partition = entry.getKey(); - long fetchOffset = data.sessionPartitions().get(partition).fetchOffset; - FetchResponse.PartitionData fetchData = entry.getValue(); - - log.debug("Fetch {} at offset {} for partition {} returned fetch data {}", - isolationLevel, fetchOffset, partition, fetchData); - completedFetches.add(new CompletedFetch(partition, fetchOffset, fetchData, metricAggregator, - resp.requestHeader().apiVersion())); - } - - sensors.fetchLatency.record(resp.requestLatencyMs()); } @Override public void onFailure(RuntimeException e) { - FetchSessionHandler handler = sessionHandlers.get(fetchTarget.id()); - if (handler != null) { - handler.handleError(e); + synchronized (Fetcher.this) { + FetchSessionHandler handler = sessionHandler(fetchTarget.id()); + if (handler != null) { + handler.handleError(e); + } } } }); @@ -313,7 +331,7 @@ else if (error.exception() instanceof RetriableException) if (!shouldRetry) { HashMap> topicsPartitionInfos = new HashMap<>(); for (String topic : cluster.topics()) - topicsPartitionInfos.put(topic, cluster.availablePartitionsForTopic(topic)); + topicsPartitionInfos.put(topic, cluster.partitionsForTopic(topic)); return topicsPartitionInfos; } } @@ -422,7 +440,7 @@ private ListOffsetResult fetchOffsetsByTimes(Map timestamp if (value.partitionsToRetry.isEmpty()) return result; - remainingToSearch.keySet().removeAll(result.fetchedOffsets.keySet()); + remainingToSearch.keySet().retainAll(value.partitionsToRetry); } else if (!future.isRetriable()) { throw future.exception(); } @@ -590,7 +608,8 @@ private void resetOffsetsAsync(Map partitionResetTimestamp for (TopicPartition tp : partitionResetTimestamps.keySet()) metadata.add(tp.topic()); - Map> timestampsToSearchByNode = groupListOffsetRequests(partitionResetTimestamps); + Map> timestampsToSearchByNode = + groupListOffsetRequests(partitionResetTimestamps, new HashSet<>()); for (Map.Entry> entry : timestampsToSearchByNode.entrySet()) { Node node = entry.getKey(); final Map resetTimestamps = entry.getValue(); @@ -639,18 +658,19 @@ private RequestFuture sendListOffsetsRequests(final Map> timestampsToSearchByNode = groupListOffsetRequests(timestampsToSearch); + final Set partitionsToRetry = new HashSet<>(); + Map> timestampsToSearchByNode = + groupListOffsetRequests(timestampsToSearch, partitionsToRetry); if (timestampsToSearchByNode.isEmpty()) return RequestFuture.failure(new StaleMetadataException()); final RequestFuture listOffsetRequestsFuture = new RequestFuture<>(); final Map fetchedTimestampOffsets = new HashMap<>(); - final Set partitionsToRetry = new HashSet<>(); final AtomicInteger remainingResponses = new AtomicInteger(timestampsToSearchByNode.size()); for (Map.Entry> entry : timestampsToSearchByNode.entrySet()) { RequestFuture future = - sendListOffsetRequest(entry.getKey(), entry.getValue(), requireTimestamps); + sendListOffsetRequest(entry.getKey(), entry.getValue(), requireTimestamps); future.addListener(new RequestFutureListener() { @Override public void onSuccess(ListOffsetResult partialResult) { @@ -677,7 +697,16 @@ public void onFailure(RuntimeException e) { return listOffsetRequestsFuture; } - private Map> groupListOffsetRequests(Map timestampsToSearch) { + /** + * Groups timestamps to search by node for topic partitions in `timestampsToSearch` that have + * leaders available. Topic partitions from `timestampsToSearch` that do not have their leader + * available are added to `partitionsToRetry` + * @param timestampsToSearch The mapping from partitions ot the target timestamps + * @param partitionsToRetry A set of topic partitions that will be extended with partitions + * that need metadata update or re-connect to the leader. + */ + private Map> groupListOffsetRequests( + Map timestampsToSearch, Set partitionsToRetry) { final Map> timestampsToSearchByNode = new HashMap<>(); for (Map.Entry entry: timestampsToSearch.entrySet()) { TopicPartition tp = entry.getKey(); @@ -686,9 +715,11 @@ private Map> groupListOffsetRequests(Map> groupListOffsetRequests(Map topicData = timestampsToSearchByNode.get(node); @@ -866,7 +898,7 @@ private Map prepareFetchRequests() { // if there is a leader and no in-flight requests, issue a new fetch FetchSessionHandler.Builder builder = fetchable.get(node); if (builder == null) { - FetchSessionHandler handler = sessionHandlers.get(node.id()); + FetchSessionHandler handler = sessionHandler(node.id()); if (handler == null) { handler = new FetchSessionHandler(logContext, node.id()); sessionHandlers.put(node.id(), handler); @@ -1023,6 +1055,11 @@ public void onAssignment(Set assignment) { sensors.updatePartitionLagAndLeadSensors(assignment); } + // Visibilty for testing + protected FetchSessionHandler sessionHandler(int node) { + return sessionHandlers.get(node); + } + public static Sensor throttleTimeSensor(Metrics metrics, FetcherMetricsRegistry metricsRegistry) { Sensor fetchThrottleTimeSensor = metrics.sensor("fetch-throttle-time"); fetchThrottleTimeSensor.add(metrics.metricInstance(metricsRegistry.fetchThrottleTimeAvg), new Avg()); @@ -1242,8 +1279,7 @@ private boolean containsAbortMarker(RecordBatch batch) { Iterator batchIterator = batch.iterator(); if (!batchIterator.hasNext()) - throw new InvalidRecordException("Invalid batch for partition " + partition + " at offset " + - batch.baseOffset() + " with control sequence set, but no records"); + return false; Record firstRecord = batchIterator.next(); return ControlRecordType.ABORT == ControlRecordType.parse(firstRecord.key()); @@ -1447,16 +1483,6 @@ private void recordPartitionLag(TopicPartition tp, long lag) { recordsLag.add(this.metrics.metricInstance(metricsRegistry.partitionRecordsLag, metricTags), new Value()); recordsLag.add(this.metrics.metricInstance(metricsRegistry.partitionRecordsLagMax, metricTags), new Max()); recordsLag.add(this.metrics.metricInstance(metricsRegistry.partitionRecordsLagAvg, metricTags), new Avg()); - - recordsLag.add(this.metrics.metricName(name, - metricsRegistry.partitionRecordsLagDeprecated.group(), - metricsRegistry.partitionRecordsLagDeprecated.description()), new Value()); - recordsLag.add(this.metrics.metricName(name + "-max", - metricsRegistry.partitionRecordsLagMaxDeprecated.group(), - metricsRegistry.partitionRecordsLagMaxDeprecated.description()), new Max()); - recordsLag.add(this.metrics.metricName(name + "-avg", - metricsRegistry.partitionRecordsLagAvgDeprecated.group(), - metricsRegistry.partitionRecordsLagAvgDeprecated.description()), new Avg()); } recordsLag.record(lag); } diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/FetcherMetricsRegistry.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/FetcherMetricsRegistry.java index 6eb4fa20ff48f..f86961545cc99 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/FetcherMetricsRegistry.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/FetcherMetricsRegistry.java @@ -54,10 +54,6 @@ public class FetcherMetricsRegistry { public MetricNameTemplate partitionRecordsLead; public MetricNameTemplate partitionRecordsLeadMin; public MetricNameTemplate partitionRecordsLeadAvg; - // To remove in 2.0 - public MetricNameTemplate partitionRecordsLagDeprecated; - public MetricNameTemplate partitionRecordsLagMaxDeprecated; - public MetricNameTemplate partitionRecordsLagAvgDeprecated; public FetcherMetricsRegistry() { this(new HashSet(), ""); @@ -68,72 +64,65 @@ public FetcherMetricsRegistry(String metricGrpPrefix) { } public FetcherMetricsRegistry(Set tags, String metricGrpPrefix) { - + /***** Client level *****/ String groupName = metricGrpPrefix + "-fetch-manager-metrics"; - - this.fetchSizeAvg = new MetricNameTemplate("fetch-size-avg", groupName, + + this.fetchSizeAvg = new MetricNameTemplate("fetch-size-avg", groupName, "The average number of bytes fetched per request", tags); - this.fetchSizeMax = new MetricNameTemplate("fetch-size-max", groupName, + this.fetchSizeMax = new MetricNameTemplate("fetch-size-max", groupName, "The maximum number of bytes fetched per request", tags); - this.bytesConsumedRate = new MetricNameTemplate("bytes-consumed-rate", groupName, + this.bytesConsumedRate = new MetricNameTemplate("bytes-consumed-rate", groupName, "The average number of bytes consumed per second", tags); this.bytesConsumedTotal = new MetricNameTemplate("bytes-consumed-total", groupName, "The total number of bytes consumed", tags); - this.recordsPerRequestAvg = new MetricNameTemplate("records-per-request-avg", groupName, + this.recordsPerRequestAvg = new MetricNameTemplate("records-per-request-avg", groupName, "The average number of records in each request", tags); - this.recordsConsumedRate = new MetricNameTemplate("records-consumed-rate", groupName, + this.recordsConsumedRate = new MetricNameTemplate("records-consumed-rate", groupName, "The average number of records consumed per second", tags); this.recordsConsumedTotal = new MetricNameTemplate("records-consumed-total", groupName, "The total number of records consumed", tags); - this.fetchLatencyAvg = new MetricNameTemplate("fetch-latency-avg", groupName, + this.fetchLatencyAvg = new MetricNameTemplate("fetch-latency-avg", groupName, "The average time taken for a fetch request.", tags); - this.fetchLatencyMax = new MetricNameTemplate("fetch-latency-max", groupName, + this.fetchLatencyMax = new MetricNameTemplate("fetch-latency-max", groupName, "The max time taken for any fetch request.", tags); - this.fetchRequestRate = new MetricNameTemplate("fetch-rate", groupName, + this.fetchRequestRate = new MetricNameTemplate("fetch-rate", groupName, "The number of fetch requests per second.", tags); this.fetchRequestTotal = new MetricNameTemplate("fetch-total", groupName, "The total number of fetch requests.", tags); - this.recordsLagMax = new MetricNameTemplate("records-lag-max", groupName, + this.recordsLagMax = new MetricNameTemplate("records-lag-max", groupName, "The maximum lag in terms of number of records for any partition in this window", tags); this.recordsLeadMin = new MetricNameTemplate("records-lead-min", groupName, "The minimum lead in terms of number of records for any partition in this window", tags); - this.fetchThrottleTimeAvg = new MetricNameTemplate("fetch-throttle-time-avg", groupName, + this.fetchThrottleTimeAvg = new MetricNameTemplate("fetch-throttle-time-avg", groupName, "The average throttle time in ms", tags); - this.fetchThrottleTimeMax = new MetricNameTemplate("fetch-throttle-time-max", groupName, + this.fetchThrottleTimeMax = new MetricNameTemplate("fetch-throttle-time-max", groupName, "The maximum throttle time in ms", tags); /***** Topic level *****/ Set topicTags = new LinkedHashSet<>(tags); topicTags.add("topic"); - this.topicFetchSizeAvg = new MetricNameTemplate("fetch-size-avg", groupName, + this.topicFetchSizeAvg = new MetricNameTemplate("fetch-size-avg", groupName, "The average number of bytes fetched per request for a topic", topicTags); - this.topicFetchSizeMax = new MetricNameTemplate("fetch-size-max", groupName, + this.topicFetchSizeMax = new MetricNameTemplate("fetch-size-max", groupName, "The maximum number of bytes fetched per request for a topic", topicTags); - this.topicBytesConsumedRate = new MetricNameTemplate("bytes-consumed-rate", groupName, + this.topicBytesConsumedRate = new MetricNameTemplate("bytes-consumed-rate", groupName, "The average number of bytes consumed per second for a topic", topicTags); this.topicBytesConsumedTotal = new MetricNameTemplate("bytes-consumed-total", groupName, "The total number of bytes consumed for a topic", topicTags); - this.topicRecordsPerRequestAvg = new MetricNameTemplate("records-per-request-avg", groupName, + this.topicRecordsPerRequestAvg = new MetricNameTemplate("records-per-request-avg", groupName, "The average number of records in each request for a topic", topicTags); - this.topicRecordsConsumedRate = new MetricNameTemplate("records-consumed-rate", groupName, + this.topicRecordsConsumedRate = new MetricNameTemplate("records-consumed-rate", groupName, "The average number of records consumed per second for a topic", topicTags); this.topicRecordsConsumedTotal = new MetricNameTemplate("records-consumed-total", groupName, "The total number of records consumed for a topic", topicTags); - - this.partitionRecordsLagDeprecated = new MetricNameTemplate("{topic}-{partition}.records-lag", groupName, - "The latest lag of the partition (DEPRECATED use the tag based version instead)", tags); - this.partitionRecordsLagMaxDeprecated = new MetricNameTemplate("{topic}-{partition}.records-lag-max", groupName, - "The max lag of the partition (DEPRECATED use the tag based version instead)", tags); - this.partitionRecordsLagAvgDeprecated = new MetricNameTemplate("{topic}-{partition}.records-lag-avg", groupName, - "The average lag of the partition (DEPRECATED use the tag based version instead)", tags); /***** Partition level *****/ Set partitionTags = new HashSet<>(topicTags); @@ -150,9 +139,9 @@ public FetcherMetricsRegistry(Set tags, String metricGrpPrefix) { "The min lead of the partition", partitionTags); this.partitionRecordsLeadAvg = new MetricNameTemplate("records-lead-avg", groupName, "The average lead of the partition", partitionTags); - + } - + public List getAllTemplates() { return Arrays.asList( fetchSizeAvg, @@ -177,9 +166,6 @@ public List getAllTemplates() { topicRecordsPerRequestAvg, topicRecordsConsumedRate, topicRecordsConsumedTotal, - partitionRecordsLagDeprecated, - partitionRecordsLagAvgDeprecated, - partitionRecordsLagMaxDeprecated, partitionRecordsLag, partitionRecordsLagAvg, partitionRecordsLagMax, 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 index 38a7c78a599c4..01d78101a1e21 100644 --- 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 @@ -20,9 +20,9 @@ * A helper class for managing the heartbeat to the coordinator */ public final class Heartbeat { - private final long sessionTimeout; - private final long heartbeatInterval; - private final long maxPollInterval; + private final int sessionTimeoutMs; + private final int heartbeatIntervalMs; + private final int maxPollIntervalMs; private final long retryBackoffMs; private volatile long lastHeartbeatSend; // volatile since it is read by metrics @@ -31,16 +31,16 @@ public final class Heartbeat { private long lastPoll; private boolean heartbeatFailed; - public Heartbeat(long sessionTimeout, - long heartbeatInterval, - long maxPollInterval, + public Heartbeat(int sessionTimeoutMs, + int heartbeatIntervalMs, + int maxPollIntervalMs, long retryBackoffMs) { - if (heartbeatInterval >= sessionTimeout) + if (heartbeatIntervalMs >= sessionTimeoutMs) throw new IllegalArgumentException("Heartbeat must be set lower than the session timeout"); - this.sessionTimeout = sessionTimeout; - this.heartbeatInterval = heartbeatInterval; - this.maxPollInterval = maxPollInterval; + this.sessionTimeoutMs = sessionTimeoutMs; + this.heartbeatIntervalMs = heartbeatIntervalMs; + this.maxPollIntervalMs = maxPollIntervalMs; this.retryBackoffMs = retryBackoffMs; } @@ -75,7 +75,7 @@ public long timeToNextHeartbeat(long now) { if (heartbeatFailed) delayToNextHeartbeat = retryBackoffMs; else - delayToNextHeartbeat = heartbeatInterval; + delayToNextHeartbeat = heartbeatIntervalMs; if (timeSinceLastHeartbeat > delayToNextHeartbeat) return 0; @@ -84,11 +84,11 @@ public long timeToNextHeartbeat(long now) { } public boolean sessionTimeoutExpired(long now) { - return now - Math.max(lastSessionReset, lastHeartbeatReceive) > sessionTimeout; + return now - Math.max(lastSessionReset, lastHeartbeatReceive) > sessionTimeoutMs; } public long interval() { - return heartbeatInterval; + return heartbeatIntervalMs; } public void resetTimeouts(long now) { @@ -98,7 +98,11 @@ public void resetTimeouts(long now) { } public boolean pollTimeoutExpired(long now) { - return now - lastPoll > maxPollInterval; + return now - lastPoll > maxPollIntervalMs; + } + + public long lastPollTime() { + return lastPoll; } } \ No newline at end of file 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 index e28973452a7f8..542c413d33e55 100644 --- 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 @@ -268,6 +268,14 @@ public Set assignedPartitions() { return this.assignment.partitionSet(); } + /** + * Provides the number of assigned partitions in a thread safe manner. + * @return the number of assigned partitions. + */ + public int numAssignedPartitions() { + return this.assignment.size(); + } + public List fetchablePartitions() { List fetchable = new ArrayList<>(assignment.size()); for (PartitionStates.PartitionState state : assignment.partitionStates()) { 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 a5af5b60093d9..f18f786a03c20 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 @@ -66,6 +66,7 @@ import org.apache.kafka.common.utils.KafkaThread; import org.apache.kafka.common.utils.LogContext; import org.apache.kafka.common.utils.Time; +import org.apache.kafka.common.utils.Utils; import org.slf4j.Logger; import java.net.InetSocketAddress; @@ -789,12 +790,12 @@ public Future send(ProducerRecord record) { * * @throws AuthenticationException if authentication fails. See the exception for more details * @throws AuthorizationException fatal error indicating that the producer is not allowed to write - * @throws IllegalStateException if a transactional.id has been configured and no transaction has been started + * @throws IllegalStateException if a transactional.id has been configured and no transaction has been started, or + * when send is invoked after producer has been closed. * @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 TimeoutException If the time taken for fetching metadata or allocating memory for the record has surpassed max.block.ms. * @throws KafkaException If a Kafka related error occurs that does not belong to the public API exceptions. - * */ @Override public Future send(ProducerRecord record, Callback callback) { @@ -803,14 +804,29 @@ public Future send(ProducerRecord record, Callback callbac return doSend(interceptedRecord, callback); } + // Verify that this producer instance has not been closed. This method throws IllegalStateException if the producer + // has already been closed. + private void throwIfProducerClosed() { + if (ioThread == null || !ioThread.isAlive()) + throw new IllegalStateException("Cannot perform operation after producer has been closed"); + } + /** * Implementation of asynchronously send a record to a topic. */ private Future doSend(ProducerRecord record, Callback callback) { TopicPartition tp = null; try { + throwIfProducerClosed(); // first make sure the metadata for the topic is available - ClusterAndWaitTime clusterAndWaitTime = waitOnMetadata(record.topic(), record.partition(), maxBlockTimeMs); + ClusterAndWaitTime clusterAndWaitTime; + try { + clusterAndWaitTime = waitOnMetadata(record.topic(), record.partition(), maxBlockTimeMs); + } catch (KafkaException e) { + if (metadata.isClosed()) + throw new KafkaException("Producer closed while send in progress", e); + throw e; + } long remainingWaitMs = Math.max(0, maxBlockTimeMs - clusterAndWaitTime.waitedOnMetadataMs); Cluster cluster = clusterAndWaitTime.cluster; byte[] serializedKey; @@ -895,6 +911,7 @@ private void setReadOnly(Headers headers) { * @param partition A specific partition expected to exist in metadata, or null if there's no preference * @param maxWaitMs The maximum time in ms for waiting on the metadata * @return The cluster containing topic metadata and the amount of time we waited in ms + * @throws KafkaException for all Kafka-related exceptions, including the case where this method is called after producer close */ private ClusterAndWaitTime waitOnMetadata(String topic, Integer partition, long maxWaitMs) throws InterruptedException { // add topic to metadata topic list if it is not there already and reset expiry @@ -909,12 +926,15 @@ private ClusterAndWaitTime waitOnMetadata(String topic, Integer partition, long long begin = time.milliseconds(); long remainingWaitMs = maxWaitMs; long elapsed; - // Issue metadata requests until we have metadata for the topic or maxWaitTimeMs is exceeded. - // In case we already have cached metadata for the topic, but the requested partition is greater - // than expected, issue an update request only once. This is necessary in case the metadata + // Issue metadata requests until we have metadata for the topic and the requested partition, + // or until maxWaitTimeMs is exceeded. This is necessary in case the metadata // is stale and the number of partitions for this topic has increased in the meantime. do { - log.trace("Requesting metadata update for topic {}.", topic); + if (partition != null) { + log.trace("Requesting metadata update for partition {} of topic {}.", partition, topic); + } else { + log.trace("Requesting metadata update for topic {}.", topic); + } metadata.add(topic); int version = metadata.requestUpdate(); sender.wakeup(); @@ -922,22 +942,24 @@ private ClusterAndWaitTime waitOnMetadata(String topic, Integer partition, long metadata.awaitUpdate(version, remainingWaitMs); } catch (TimeoutException ex) { // Rethrow with original maxWaitMs to prevent logging exception with remainingWaitMs - throw new TimeoutException("Failed to update metadata after " + maxWaitMs + " ms."); + throw new TimeoutException( + String.format("Topic %s not present in metadata after %d ms.", + topic, maxWaitMs)); } cluster = metadata.fetch(); elapsed = time.milliseconds() - begin; - if (elapsed >= maxWaitMs) - throw new TimeoutException("Failed to update metadata after " + maxWaitMs + " ms."); + if (elapsed >= maxWaitMs) { + throw new TimeoutException(partitionsCount == null ? + String.format("Topic %s not present in metadata after %d ms.", + topic, maxWaitMs) : + String.format("Partition %d of topic %s with partition count %d is not present in metadata after %d ms.", + partition, topic, partitionsCount, maxWaitMs)); + } if (cluster.unauthorizedTopics().contains(topic)) throw new TopicAuthorizationException(topic); remainingWaitMs = maxWaitMs - elapsed; partitionsCount = cluster.partitionCountForTopic(topic); - } while (partitionsCount == null); - - if (partition != null && partition >= partitionsCount) { - throw new KafkaException( - String.format("Invalid partition given with record: %d is not in the range [0...%d).", partition, partitionsCount)); - } + } while (partitionsCount == null || (partition != null && partition >= partitionsCount)); return new ClusterAndWaitTime(cluster, elapsed); } @@ -1008,8 +1030,9 @@ public void flush() { * Get the partition metadata for the given topic. This can be used for custom partitioning. * @throws AuthenticationException if authentication fails. See the exception for more details * @throws AuthorizationException if not authorized to the specified topic. See the exception for more details - * @throws InterruptException If the thread is interrupted while blocked + * @throws InterruptException if the thread is interrupted while blocked * @throws TimeoutException if metadata could not be refreshed within {@code max.block.ms} + * @throws KafkaException for all Kafka-related exceptions, including the case where this method is called after producer close */ @Override public List partitionsFor(String topic) { @@ -1107,11 +1130,11 @@ private void close(long timeout, TimeUnit timeUnit, boolean swallowException) { } } - ClientUtils.closeQuietly(interceptors, "producer interceptors", firstException); - ClientUtils.closeQuietly(metrics, "producer metrics", firstException); - ClientUtils.closeQuietly(keySerializer, "producer keySerializer", firstException); - ClientUtils.closeQuietly(valueSerializer, "producer valueSerializer", firstException); - ClientUtils.closeQuietly(partitioner, "producer partitioner", firstException); + Utils.closeQuietly(interceptors, "producer interceptors", firstException); + Utils.closeQuietly(metrics, "producer metrics", firstException); + Utils.closeQuietly(keySerializer, "producer keySerializer", firstException); + Utils.closeQuietly(valueSerializer, "producer valueSerializer", firstException); + Utils.closeQuietly(partitioner, "producer partitioner", firstException); AppInfoParser.unregisterAppInfo(JMX_PREFIX, clientId, metrics); log.debug("Kafka producer has been closed"); Throwable exception = firstException.get(); 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 9e9869a7e488c..ab036a0992cb1 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 @@ -21,6 +21,7 @@ import org.apache.kafka.clients.producer.internals.FutureRecordMetadata; import org.apache.kafka.clients.producer.internals.ProduceRequestResult; 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; @@ -67,6 +68,7 @@ public class MockProducer implements Producer { private boolean transactionCommitted; private boolean transactionAborted; private boolean producerFenced; + private boolean producerFencedOnClose; private boolean sentOffsets; private long commitCount = 0L; private Map mockMetrics; @@ -206,7 +208,7 @@ public void abortTransaction() throws ProducerFencedException { this.transactionInFlight = false; } - private void verifyProducerState() { + private synchronized void verifyProducerState() { if (this.closed) { throw new IllegalStateException("MockProducer is already closed."); } @@ -244,7 +246,12 @@ public synchronized Future send(ProducerRecord record) { */ @Override public synchronized Future send(ProducerRecord record, Callback callback) { - verifyProducerState(); + if (this.closed) { + throw new IllegalStateException("MockProducer is already closed."); + } + if (this.producerFenced) { + throw new KafkaException("MockProducer is fenced.", new ProducerFencedException("Fenced")); + } int partition = 0; if (!this.cluster.partitionsForTopic(record.topic()).isEmpty()) partition = partition(record, this.cluster); @@ -311,8 +318,8 @@ public void close() { @Override public void close(long timeout, TimeUnit timeUnit) { - if (this.closed) { - throw new IllegalStateException("MockProducer is already closed."); + if (producerFencedOnClose) { + throw new ProducerFencedException("MockProducer is fenced."); } this.closed = true; } @@ -321,12 +328,18 @@ public boolean closed() { return this.closed; } - public void fenceProducer() { + public synchronized void fenceProducer() { verifyProducerState(); verifyTransactionsInitialized(); this.producerFenced = true; } + public void fenceProducerOnClose() { + verifyProducerState(); + verifyTransactionsInitialized(); + this.producerFencedOnClose = true; + } + public boolean transactionInitialized() { return this.transactionInitialized; } diff --git a/clients/src/main/java/org/apache/kafka/clients/producer/RecordMetadata.java b/clients/src/main/java/org/apache/kafka/clients/producer/RecordMetadata.java index 0924244829b6f..1ae5cf654b8de 100644 --- a/clients/src/main/java/org/apache/kafka/clients/producer/RecordMetadata.java +++ b/clients/src/main/java/org/apache/kafka/clients/producer/RecordMetadata.java @@ -56,16 +56,6 @@ public RecordMetadata(TopicPartition topicPartition, long baseOffset, long relat this.topicPartition = topicPartition; } - /** - * @deprecated As of 0.11.0. Use @{@link RecordMetadata#RecordMetadata(TopicPartition, long, long, long, Long, int, int)}. - */ - @Deprecated - public RecordMetadata(TopicPartition topicPartition, long baseOffset, long relativeOffset, long timestamp, - long checksum, int serializedKeySize, int serializedValueSize) { - this(topicPartition, baseOffset, relativeOffset, timestamp, Long.valueOf(checksum), serializedKeySize, - serializedValueSize); - } - /** * Indicates whether the record metadata includes the offset. * @return true if the offset is included in the metadata, false otherwise. 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 e2b58448dc92e..31c6d754c9d92 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 @@ -19,6 +19,7 @@ import org.apache.kafka.clients.ApiVersions; import org.apache.kafka.clients.producer.Callback; import org.apache.kafka.common.Cluster; +import org.apache.kafka.common.KafkaException; import org.apache.kafka.common.MetricName; import org.apache.kafka.common.Node; import org.apache.kafka.common.PartitionInfo; @@ -195,7 +196,7 @@ public RecordAppendResult append(TopicPartition tp, Deque dq = getOrCreateDeque(tp); synchronized (dq) { if (closed) - throw new IllegalStateException("Cannot send after the producer is closed."); + throw new KafkaException("Producer closed while send in progress"); RecordAppendResult appendResult = tryAppend(timestamp, key, value, headers, callback, dq); if (appendResult != null) return appendResult; @@ -209,7 +210,7 @@ public RecordAppendResult append(TopicPartition tp, 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."); + throw new KafkaException("Producer closed while send in progress"); RecordAppendResult appendResult = tryAppend(timestamp, key, value, headers, callback, dq); if (appendResult != null) { @@ -700,7 +701,7 @@ public void abortIncompleteBatches() { * Go through incomplete batches and abort them. */ private void abortBatches() { - abortBatches(new IllegalStateException("Producer is closed forcefully.")); + abortBatches(new KafkaException("Producer is closed forcefully.")); } /** 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 7c941796ce453..a0fcad071ffe0 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 @@ -109,7 +109,7 @@ public class Sender implements Runnable { private final SenderMetrics sensors; /* the max time to wait for the server to respond to the request*/ - private final int requestTimeout; + private final int requestTimeoutMs; /* The max time to wait before retrying a request which has failed */ private final long retryBackoffMs; @@ -130,7 +130,7 @@ public Sender(LogContext logContext, int retries, SenderMetricsRegistry metricsRegistry, Time time, - int requestTimeout, + int requestTimeoutMs, long retryBackoffMs, TransactionManager transactionManager, ApiVersions apiVersions) { @@ -145,7 +145,7 @@ public Sender(LogContext logContext, this.retries = retries; this.time = time; this.sensors = new SenderMetrics(metricsRegistry); - this.requestTimeout = requestTimeout; + this.requestTimeoutMs = requestTimeoutMs; this.retryBackoffMs = retryBackoffMs; this.apiVersions = apiVersions; this.transactionManager = transactionManager; @@ -280,7 +280,7 @@ private long sendProducerData(long now) { } } - List expiredBatches = this.accumulator.expiredBatches(this.requestTimeout, now); + List expiredBatches = this.accumulator.expiredBatches(this.requestTimeoutMs, now); // Reset the producer id if an expired batch has previously been sent to the broker. Also update the metrics // for expired batches. see the documentation of @TransactionState.resetProducerId to understand why // we need to reset the producer id here. @@ -342,12 +342,12 @@ private boolean maybeSendTransactionalRequest(long now) { break; } - if (!NetworkClientUtils.awaitReady(client, targetNode, time, requestTimeout)) { + if (!NetworkClientUtils.awaitReady(client, targetNode, time, requestTimeoutMs)) { transactionManager.lookupCoordinator(nextRequestHandler); break; } } else { - targetNode = awaitLeastLoadedNodeReady(requestTimeout); + targetNode = awaitLeastLoadedNodeReady(requestTimeoutMs); } if (targetNode != null) { @@ -355,7 +355,7 @@ private boolean maybeSendTransactionalRequest(long now) { time.sleep(nextRequestHandler.retryBackoffMs()); ClientRequest clientRequest = client.newClientRequest(targetNode.idString(), - requestBuilder, now, true, nextRequestHandler); + requestBuilder, now, true, requestTimeoutMs, nextRequestHandler); transactionManager.setInFlightTransactionalRequestCorrelationId(clientRequest.correlationId()); log.debug("Sending transactional request {} to node {}", requestBuilder, targetNode); @@ -409,7 +409,7 @@ public void forceClose() { private ClientResponse sendAndAwaitInitProducerIdRequest(Node node) throws IOException { String nodeId = node.idString(); InitProducerIdRequest.Builder builder = new InitProducerIdRequest.Builder(null); - ClientRequest request = client.newClientRequest(nodeId, builder, time.milliseconds(), true, null); + ClientRequest request = client.newClientRequest(nodeId, builder, time.milliseconds(), true, requestTimeoutMs, null); return NetworkClientUtils.sendAndReceive(client, request, time); } @@ -422,9 +422,9 @@ private Node awaitLeastLoadedNodeReady(long remainingTimeMs) throws IOException } private void maybeWaitForProducerId() { - while (!transactionManager.hasProducerId() && !transactionManager.hasError()) { + while (!forceClose && !transactionManager.hasProducerId() && !transactionManager.hasError()) { try { - Node node = awaitLeastLoadedNodeReady(requestTimeout); + Node node = awaitLeastLoadedNodeReady(requestTimeoutMs); if (node != null) { ClientResponse response = sendAndAwaitInitProducerIdRequest(node); InitProducerIdResponse initProducerIdResponse = (InitProducerIdResponse) response.responseBody(); @@ -652,7 +652,7 @@ private boolean canRetry(ProducerBatch batch, ProduceResponse.PartitionResponse */ private void sendProduceRequests(Map> collated, long now) { for (Map.Entry> entry : collated.entrySet()) - sendProduceRequest(now, entry.getKey(), acks, requestTimeout, entry.getValue()); + sendProduceRequest(now, entry.getKey(), acks, requestTimeoutMs, entry.getValue()); } /** @@ -702,7 +702,8 @@ public void onComplete(ClientResponse response) { }; String nodeId = Integer.toString(destination); - ClientRequest clientRequest = client.newClientRequest(nodeId, requestBuilder, now, acks != 0, callback); + ClientRequest clientRequest = client.newClientRequest(nodeId, requestBuilder, now, acks != 0, + requestTimeoutMs, callback); client.send(clientRequest, now); log.trace("Sent produce request to {}: {}", nodeId, requestBuilder); } diff --git a/clients/src/main/java/org/apache/kafka/clients/producer/internals/TransactionManager.java b/clients/src/main/java/org/apache/kafka/clients/producer/internals/TransactionManager.java index b242d5a65a639..124b5f8c68b37 100644 --- a/clients/src/main/java/org/apache/kafka/clients/producer/internals/TransactionManager.java +++ b/clients/src/main/java/org/apache/kafka/clients/producer/internals/TransactionManager.java @@ -26,6 +26,7 @@ import org.apache.kafka.common.errors.GroupAuthorizationException; import org.apache.kafka.common.errors.TopicAuthorizationException; import org.apache.kafka.common.protocol.Errors; +import org.apache.kafka.common.record.DefaultRecordBatch; import org.apache.kafka.common.record.RecordBatch; import org.apache.kafka.common.requests.AbstractRequest; import org.apache.kafka.common.requests.AbstractResponse; @@ -419,7 +420,7 @@ synchronized void incrementSequenceNumber(TopicPartition topicPartition, int inc if (currentSequenceNumber == null) throw new IllegalStateException("Attempt to increment sequence number for a partition with no current sequence."); - currentSequenceNumber += increment; + currentSequenceNumber = DefaultRecordBatch.incrementSequence(currentSequenceNumber, increment); nextSequence.put(topicPartition, currentSequenceNumber); } @@ -1272,50 +1273,49 @@ String coordinatorKey() { public void handleResponse(AbstractResponse response) { TxnOffsetCommitResponse txnOffsetCommitResponse = (TxnOffsetCommitResponse) response; boolean coordinatorReloaded = false; - boolean hadFailure = false; Map errors = txnOffsetCommitResponse.errors(); + log.debug("Received TxnOffsetCommit response for consumer group {}: {}", builder.consumerGroupId(), + errors); + for (Map.Entry entry : errors.entrySet()) { TopicPartition topicPartition = entry.getKey(); Errors error = entry.getValue(); if (error == Errors.NONE) { - log.debug("Successfully added offsets {} from consumer group {} to transaction.", - builder.offsets(), builder.consumerGroupId()); pendingTxnOffsetCommits.remove(topicPartition); } else if (error == Errors.COORDINATOR_NOT_AVAILABLE || error == Errors.NOT_COORDINATOR || error == Errors.REQUEST_TIMED_OUT) { - hadFailure = true; if (!coordinatorReloaded) { coordinatorReloaded = true; lookupCoordinator(FindCoordinatorRequest.CoordinatorType.GROUP, builder.consumerGroupId()); } - } else if (error == Errors.UNKNOWN_TOPIC_OR_PARTITION) { - hadFailure = true; + } else if (error == Errors.UNKNOWN_TOPIC_OR_PARTITION + || error == Errors.COORDINATOR_LOAD_IN_PROGRESS) { + // If the topic is unknown or the coordinator is loading, retry with the current coordinator + continue; } else if (error == Errors.GROUP_AUTHORIZATION_FAILED) { abortableError(new GroupAuthorizationException(builder.consumerGroupId())); - return; + break; } else if (error == Errors.TRANSACTIONAL_ID_AUTHORIZATION_FAILED || error == Errors.INVALID_PRODUCER_EPOCH || error == Errors.UNSUPPORTED_FOR_MESSAGE_FORMAT) { fatalError(error.exception()); - return; + break; } else { fatalError(new KafkaException("Unexpected error in TxnOffsetCommitResponse: " + error.message())); - return; + break; } } - if (!hadFailure || !result.isSuccessful()) { - // all attempted partitions were either successful, or there was a fatal failure. - // either way, we are not retrying, so complete the request. + if (result.isCompleted()) { + pendingTxnOffsetCommits.clear(); + } else if (pendingTxnOffsetCommits.isEmpty()) { result.done(); - return; - } - - // retry the commits which failed with a retriable error. - if (!pendingTxnOffsetCommits.isEmpty()) + } else { + // Retry the commits which failed with a retriable error reenqueue(); + } } } } diff --git a/clients/src/main/java/org/apache/kafka/common/acl/AclBinding.java b/clients/src/main/java/org/apache/kafka/common/acl/AclBinding.java index d264ef1a4f627..67dbfc0545c3b 100644 --- a/clients/src/main/java/org/apache/kafka/common/acl/AclBinding.java +++ b/clients/src/main/java/org/apache/kafka/common/acl/AclBinding.java @@ -18,49 +18,61 @@ package org.apache.kafka.common.acl; import org.apache.kafka.common.annotation.InterfaceStability; +import org.apache.kafka.common.resource.PatternType; import org.apache.kafka.common.resource.Resource; +import org.apache.kafka.common.resource.ResourcePattern; import java.util.Objects; /** - * Represents a binding between a resource and an access control entry. + * Represents a binding between a resource pattern and an access control entry. * * The API for this class is still evolving and we may break compatibility in minor releases, if necessary. */ @InterfaceStability.Evolving public class AclBinding { - private final Resource resource; + private final ResourcePattern pattern; private final AccessControlEntry entry; + /** + * Create an instance of this class with the provided parameters. + * + * @param pattern non-null resource pattern. + * @param entry non-null entry + */ + public AclBinding(ResourcePattern pattern, AccessControlEntry entry) { + this.pattern = Objects.requireNonNull(pattern, "pattern"); + this.entry = Objects.requireNonNull(entry, "entry"); + } + /** * Create an instance of this class with the provided parameters. * * @param resource non-null resource * @param entry non-null entry + * @deprecated Since 2.0. Use {@link #AclBinding(ResourcePattern, AccessControlEntry)} */ + @Deprecated public AclBinding(Resource resource, AccessControlEntry entry) { - Objects.requireNonNull(resource); - this.resource = resource; - Objects.requireNonNull(entry); - this.entry = entry; + this(new ResourcePattern(resource.resourceType(), resource.name(), PatternType.LITERAL), entry); } /** - * Return true if this binding has any UNKNOWN components. + * @return true if this binding has any UNKNOWN components. */ public boolean isUnknown() { - return resource.isUnknown() || entry.isUnknown(); + return pattern.isUnknown() || entry.isUnknown(); } /** - * Return the resource for this binding. + * @return the resource pattern for this binding. */ - public Resource resource() { - return resource; + public ResourcePattern pattern() { + return pattern; } /** - * Return the access control entry for this binding. + * @return the access control entry for this binding. */ public final AccessControlEntry entry() { return entry; @@ -70,24 +82,25 @@ public final AccessControlEntry entry() { * Create a filter which matches only this AclBinding. */ public AclBindingFilter toFilter() { - return new AclBindingFilter(resource.toFilter(), entry.toFilter()); + return new AclBindingFilter(pattern.toFilter(), entry.toFilter()); } @Override public String toString() { - return "(resource=" + resource + ", entry=" + entry + ")"; + return "(pattern=" + pattern + ", entry=" + entry + ")"; } @Override public boolean equals(Object o) { - if (!(o instanceof AclBinding)) - return false; - AclBinding other = (AclBinding) o; - return resource.equals(other.resource) && entry.equals(other.entry); + if (this == o) return true; + if (o == null || getClass() != o.getClass()) return false; + AclBinding that = (AclBinding) o; + return Objects.equals(pattern, that.pattern) && + Objects.equals(entry, that.entry); } @Override public int hashCode() { - return Objects.hash(resource, entry); + return Objects.hash(pattern, entry); } } diff --git a/clients/src/main/java/org/apache/kafka/common/acl/AclBindingFilter.java b/clients/src/main/java/org/apache/kafka/common/acl/AclBindingFilter.java index 5841b5aeb9a0e..3168ec61cfcef 100644 --- a/clients/src/main/java/org/apache/kafka/common/acl/AclBindingFilter.java +++ b/clients/src/main/java/org/apache/kafka/common/acl/AclBindingFilter.java @@ -18,7 +18,9 @@ package org.apache.kafka.common.acl; import org.apache.kafka.common.annotation.InterfaceStability; +import org.apache.kafka.common.resource.PatternType; import org.apache.kafka.common.resource.ResourceFilter; +import org.apache.kafka.common.resource.ResourcePatternFilter; import java.util.Objects; @@ -29,43 +31,53 @@ */ @InterfaceStability.Evolving public class AclBindingFilter { - private final ResourceFilter resourceFilter; + private final ResourcePatternFilter patternFilter; private final AccessControlEntryFilter entryFilter; /** * A filter which matches any ACL binding. */ - public static final AclBindingFilter ANY = new AclBindingFilter(ResourceFilter.ANY, AccessControlEntryFilter.ANY); + public static final AclBindingFilter ANY = new AclBindingFilter(ResourcePatternFilter.ANY, AccessControlEntryFilter.ANY); + + /** + * Create an instance of this filter with the provided parameters. + * + * @param patternFilter non-null pattern filter + * @param entryFilter non-null access control entry filter + */ + public AclBindingFilter(ResourcePatternFilter patternFilter, AccessControlEntryFilter entryFilter) { + this.patternFilter = Objects.requireNonNull(patternFilter, "patternFilter"); + this.entryFilter = Objects.requireNonNull(entryFilter, "entryFilter"); + } /** * Create an instance of this filter with the provided parameters. * * @param resourceFilter non-null resource filter * @param entryFilter non-null access control entry filter + * @deprecated Since 2.0. Use {@link #AclBindingFilter(ResourcePatternFilter, AccessControlEntryFilter)} */ + @Deprecated public AclBindingFilter(ResourceFilter resourceFilter, AccessControlEntryFilter entryFilter) { - Objects.requireNonNull(resourceFilter); - this.resourceFilter = resourceFilter; - Objects.requireNonNull(entryFilter); - this.entryFilter = entryFilter; + this(new ResourcePatternFilter(resourceFilter.resourceType(), resourceFilter.name(), PatternType.LITERAL), entryFilter); } /** - * Return true if this filter has any UNKNOWN components. + * @return {@code true} if this filter has any UNKNOWN components. */ public boolean isUnknown() { - return resourceFilter.isUnknown() || entryFilter.isUnknown(); + return patternFilter.isUnknown() || entryFilter.isUnknown(); } /** - * Return the resource filter. + * @return the resource pattern filter. */ - public ResourceFilter resourceFilter() { - return resourceFilter; + public ResourcePatternFilter patternFilter() { + return patternFilter; } /** - * Return the access control entry filter. + * @return the access control entry filter. */ public final AccessControlEntryFilter entryFilter() { return entryFilter; @@ -73,15 +85,16 @@ public final AccessControlEntryFilter entryFilter() { @Override public String toString() { - return "(resourceFilter=" + resourceFilter + ", entryFilter=" + entryFilter + ")"; + return "(patternFilter=" + patternFilter + ", entryFilter=" + entryFilter + ")"; } @Override public boolean equals(Object o) { - if (!(o instanceof AclBindingFilter)) - return false; - AclBindingFilter other = (AclBindingFilter) o; - return resourceFilter.equals(other.resourceFilter) && entryFilter.equals(other.entryFilter); + if (this == o) return true; + if (o == null || getClass() != o.getClass()) return false; + AclBindingFilter that = (AclBindingFilter) o; + return Objects.equals(patternFilter, that.patternFilter) && + Objects.equals(entryFilter, that.entryFilter); } /** @@ -89,14 +102,14 @@ public boolean equals(Object o) { * there are no ANY or UNKNOWN fields. */ public boolean matchesAtMostOne() { - return resourceFilter.matchesAtMostOne() && entryFilter.matchesAtMostOne(); + return patternFilter.matchesAtMostOne() && entryFilter.matchesAtMostOne(); } /** * Return a string describing an ANY or UNKNOWN field, or null if there is no such field. */ public String findIndefiniteField() { - String indefinite = resourceFilter.findIndefiniteField(); + String indefinite = patternFilter.findIndefiniteField(); if (indefinite != null) return indefinite; return entryFilter.findIndefiniteField(); @@ -106,11 +119,11 @@ public String findIndefiniteField() { * Return true if the resource filter matches the binding's resource and the entry filter matches binding's entry. */ public boolean matches(AclBinding binding) { - return resourceFilter.matches(binding.resource()) && entryFilter.matches(binding.entry()); + return patternFilter.matches(binding.pattern()) && entryFilter.matches(binding.entry()); } @Override public int hashCode() { - return Objects.hash(resourceFilter, entryFilter); + return Objects.hash(patternFilter, entryFilter); } } diff --git a/clients/src/main/java/org/apache/kafka/common/config/ConfigChangeCallback.java b/clients/src/main/java/org/apache/kafka/common/config/ConfigChangeCallback.java index d4c9948bc9308..faa7d3d87074a 100644 --- a/clients/src/main/java/org/apache/kafka/common/config/ConfigChangeCallback.java +++ b/clients/src/main/java/org/apache/kafka/common/config/ConfigChangeCallback.java @@ -16,6 +16,8 @@ */ package org.apache.kafka.common.config; +import org.apache.kafka.common.config.provider.ConfigProvider; + /** * A callback passed to {@link ConfigProvider} for subscribing to changes. */ diff --git a/clients/src/main/java/org/apache/kafka/common/config/ConfigData.java b/clients/src/main/java/org/apache/kafka/common/config/ConfigData.java index 2bd0ff6b06ab3..8661ee16cbab1 100644 --- a/clients/src/main/java/org/apache/kafka/common/config/ConfigData.java +++ b/clients/src/main/java/org/apache/kafka/common/config/ConfigData.java @@ -16,6 +16,8 @@ */ package org.apache.kafka.common.config; +import org.apache.kafka.common.config.provider.ConfigProvider; + import java.util.Map; /** 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 c9efb82e061c5..064e08584c7ed 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 @@ -711,9 +711,16 @@ else if (value instanceof String) case CLASS: if (value instanceof Class) return value; - else if (value instanceof String) - return Class.forName(trimmed, true, Utils.getContextOrKafkaClassLoader()); - else + else if (value instanceof String) { + ClassLoader contextOrKafkaClassLoader = Utils.getContextOrKafkaClassLoader(); + // Use loadClass here instead of Class.forName because the name we use here may be an alias + // and not match the name of the class that gets loaded. If that happens, Class.forName can + // throw an exception. + Class klass = contextOrKafkaClassLoader.loadClass(trimmed); + // Invoke forName here with the true name of the requested class to cause class + // initialization to take place. + return Class.forName(klass.getName(), true, contextOrKafkaClassLoader); + } else throw new ConfigException(name, value, "Expected a Class instance or class name."); default: throw new IllegalStateException("Unknown type."); diff --git a/clients/src/main/java/org/apache/kafka/common/config/ConfigResource.java b/clients/src/main/java/org/apache/kafka/common/config/ConfigResource.java index da718f54d547e..5343a6bcdfd3d 100644 --- a/clients/src/main/java/org/apache/kafka/common/config/ConfigResource.java +++ b/clients/src/main/java/org/apache/kafka/common/config/ConfigResource.java @@ -17,7 +17,12 @@ package org.apache.kafka.common.config; +import java.util.Arrays; +import java.util.Collections; +import java.util.Map; import java.util.Objects; +import java.util.function.Function; +import java.util.stream.Collectors; /** * A class representing resources that have configs. @@ -28,7 +33,25 @@ public final class ConfigResource { * Type of resource. */ public enum Type { - BROKER, TOPIC, UNKNOWN; + BROKER((byte) 4), TOPIC((byte) 2), UNKNOWN((byte) 0); + + private static final Map TYPES = Collections.unmodifiableMap( + Arrays.stream(values()).collect(Collectors.toMap(Type::id, Function.identity())) + ); + + private final byte id; + + Type(final byte id) { + this.id = id; + } + + public byte id() { + return id; + } + + public static Type forId(final byte id) { + return TYPES.getOrDefault(id, UNKNOWN); + } } private final Type type; diff --git a/clients/src/main/java/org/apache/kafka/common/config/ConfigTransformer.java b/clients/src/main/java/org/apache/kafka/common/config/ConfigTransformer.java index 7c3c516b07341..c1264724d904b 100644 --- a/clients/src/main/java/org/apache/kafka/common/config/ConfigTransformer.java +++ b/clients/src/main/java/org/apache/kafka/common/config/ConfigTransformer.java @@ -16,6 +16,9 @@ */ package org.apache.kafka.common.config; +import org.apache.kafka.common.config.provider.ConfigProvider; +import org.apache.kafka.common.config.provider.FileConfigProvider; + import java.util.ArrayList; import java.util.HashMap; import java.util.HashSet; @@ -50,7 +53,7 @@ * {@link ConfigProvider#unsubscribe(String, Set, ConfigChangeCallback)} methods. */ public class ConfigTransformer { - private static final Pattern DEFAULT_PATTERN = Pattern.compile("\\$\\{(.*?):((.*?):)?(.*?)\\}"); + public static final Pattern DEFAULT_PATTERN = Pattern.compile("\\$\\{([^}]*?):(([^}]*?):)?([^}]*?)\\}"); private static final String EMPTY_PATH = ""; private final Map configProviders; @@ -77,11 +80,13 @@ public ConfigTransformerResult transform(Map configs) { // Collect the variables from the given configs that need transformation for (Map.Entry config : configs.entrySet()) { - List vars = getVars(config.getKey(), config.getValue(), DEFAULT_PATTERN); - for (ConfigVariable var : vars) { - Map> keysByPath = keysByProvider.computeIfAbsent(var.providerName, k -> new HashMap<>()); - Set keys = keysByPath.computeIfAbsent(var.path, k -> new HashSet<>()); - keys.add(var.variable); + if (config.getValue() != null) { + List vars = getVars(config.getKey(), config.getValue(), DEFAULT_PATTERN); + for (ConfigVariable var : vars) { + Map> keysByPath = keysByProvider.computeIfAbsent(var.providerName, k -> new HashMap<>()); + Set keys = keysByPath.computeIfAbsent(var.path, k -> new HashSet<>()); + keys.add(var.variable); + } } } @@ -128,6 +133,9 @@ private static List getVars(String key, String value, Pattern pa private static String replace(Map>> lookupsByProvider, String value, Pattern pattern) { + if (value == null) { + return null; + } Matcher matcher = pattern.matcher(value); StringBuilder builder = new StringBuilder(); int i = 0; diff --git a/clients/src/main/java/org/apache/kafka/common/config/ConfigTransformerResult.java b/clients/src/main/java/org/apache/kafka/common/config/ConfigTransformerResult.java index df7bea62f3759..a05669cb6c3ee 100644 --- a/clients/src/main/java/org/apache/kafka/common/config/ConfigTransformerResult.java +++ b/clients/src/main/java/org/apache/kafka/common/config/ConfigTransformerResult.java @@ -16,6 +16,8 @@ */ package org.apache.kafka.common.config; +import org.apache.kafka.common.config.provider.ConfigProvider; + import java.util.Map; /** diff --git a/clients/src/main/java/org/apache/kafka/common/config/TopicConfig.java b/clients/src/main/java/org/apache/kafka/common/config/TopicConfig.java index d6b70032626ad..fb2208c0328d1 100755 --- a/clients/src/main/java/org/apache/kafka/common/config/TopicConfig.java +++ b/clients/src/main/java/org/apache/kafka/common/config/TopicConfig.java @@ -165,4 +165,11 @@ public class TopicConfig { "the timestamp when a broker receives a message and the timestamp specified in the message. If " + "message.timestamp.type=CreateTime, a message will be rejected if the difference in timestamp " + "exceeds this threshold. This configuration is ignored if message.timestamp.type=LogAppendTime."; + + public static final String MESSAGE_DOWNCONVERSION_ENABLE_CONFIG = "message.downconversion.enable"; + public static final String MESSAGE_DOWNCONVERSION_ENABLE_DOC = "This configuration controls whether " + + "down-conversion of message formats is enabled to satisfy consume requests. When set to false, " + + "broker will not perform down-conversion for consumers expecting an older message format. The broker responds " + + "with UNSUPPORTED_VERSION error for consume requests from such older clients. This configuration" + + "does not apply to any message format conversion that might be required for replication to followers."; } diff --git a/clients/src/main/java/org/apache/kafka/common/config/ConfigProvider.java b/clients/src/main/java/org/apache/kafka/common/config/provider/ConfigProvider.java similarity index 94% rename from clients/src/main/java/org/apache/kafka/common/config/ConfigProvider.java rename to clients/src/main/java/org/apache/kafka/common/config/provider/ConfigProvider.java index 7133baaebd01a..8561511e64e30 100644 --- a/clients/src/main/java/org/apache/kafka/common/config/ConfigProvider.java +++ b/clients/src/main/java/org/apache/kafka/common/config/provider/ConfigProvider.java @@ -14,9 +14,11 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.kafka.common.config; +package org.apache.kafka.common.config.provider; import org.apache.kafka.common.Configurable; +import org.apache.kafka.common.config.ConfigChangeCallback; +import org.apache.kafka.common.config.ConfigData; import java.io.Closeable; import java.util.Set; diff --git a/clients/src/main/java/org/apache/kafka/common/config/FileConfigProvider.java b/clients/src/main/java/org/apache/kafka/common/config/provider/FileConfigProvider.java similarity index 92% rename from clients/src/main/java/org/apache/kafka/common/config/FileConfigProvider.java rename to clients/src/main/java/org/apache/kafka/common/config/provider/FileConfigProvider.java index fefc93566f389..d25183a80119a 100644 --- a/clients/src/main/java/org/apache/kafka/common/config/FileConfigProvider.java +++ b/clients/src/main/java/org/apache/kafka/common/config/provider/FileConfigProvider.java @@ -14,8 +14,12 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.kafka.common.config; +package org.apache.kafka.common.config.provider; +import org.apache.kafka.common.config.ConfigData; +import org.apache.kafka.common.config.ConfigException; + +import java.io.BufferedReader; import java.io.FileInputStream; import java.io.IOException; import java.io.InputStreamReader; @@ -93,7 +97,7 @@ public ConfigData get(String path, Set keys) { // visible for testing protected Reader reader(String path) throws IOException { - return new InputStreamReader(new FileInputStream(path), StandardCharsets.UTF_8); + return new BufferedReader(new InputStreamReader(new FileInputStream(path), StandardCharsets.UTF_8)); } public void close() { diff --git a/clients/src/main/java/org/apache/kafka/common/errors/GroupCoordinatorNotAvailableException.java b/clients/src/main/java/org/apache/kafka/common/errors/GroupCoordinatorNotAvailableException.java deleted file mode 100644 index 03a7719a6f428..0000000000000 --- a/clients/src/main/java/org/apache/kafka/common/errors/GroupCoordinatorNotAvailableException.java +++ /dev/null @@ -1,47 +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.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. - * - * @deprecated As of Kafka 0.11, this has been replaced by {@link CoordinatorNotAvailableException} - */ -@Deprecated -public class GroupCoordinatorNotAvailableException extends RetriableException { - public static final GroupCoordinatorNotAvailableException INSTANCE = new GroupCoordinatorNotAvailableException(); - - private static final long serialVersionUID = 1L; - - public GroupCoordinatorNotAvailableException() { - super(); - } - - public GroupCoordinatorNotAvailableException(String message) { - super(message); - } - - public GroupCoordinatorNotAvailableException(String message, Throwable cause) { - super(message, cause); - } - - public GroupCoordinatorNotAvailableException(Throwable cause) { - super(cause); - } - -} diff --git a/clients/src/main/java/org/apache/kafka/common/errors/NotCoordinatorForGroupException.java b/clients/src/main/java/org/apache/kafka/common/errors/ListenerNotFoundException.java similarity index 61% rename from clients/src/main/java/org/apache/kafka/common/errors/NotCoordinatorForGroupException.java rename to clients/src/main/java/org/apache/kafka/common/errors/ListenerNotFoundException.java index cee649502e979..82c5d892f49ba 100644 --- a/clients/src/main/java/org/apache/kafka/common/errors/NotCoordinatorForGroupException.java +++ b/clients/src/main/java/org/apache/kafka/common/errors/ListenerNotFoundException.java @@ -17,30 +17,22 @@ 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. - * - * @deprecated As of Kafka 0.11, this has been replaced by {@link NotCoordinatorException} + * The leader does not have an endpoint corresponding to the listener on which metadata was requested. + * This could indicate a broker configuration error or a transient error when listeners are updated + * dynamically and client requests are processed before all brokers have updated their listeners. + * This is currently used only for missing listeners on leader brokers, but may be used for followers + * in future. */ -@Deprecated -public class NotCoordinatorForGroupException extends RetriableException { +public class ListenerNotFoundException extends InvalidMetadataException { private static final long serialVersionUID = 1L; - public NotCoordinatorForGroupException() { - super(); - } - - public NotCoordinatorForGroupException(String message) { + public ListenerNotFoundException(String message) { super(message); } - public NotCoordinatorForGroupException(String message, Throwable cause) { + public ListenerNotFoundException(String message, Throwable cause) { super(message, cause); } - public NotCoordinatorForGroupException(Throwable cause) { - super(cause); - } - } diff --git a/clients/src/main/java/org/apache/kafka/common/internals/PartitionStates.java b/clients/src/main/java/org/apache/kafka/common/internals/PartitionStates.java index 605372c6ec7e9..5b904c2671771 100644 --- a/clients/src/main/java/org/apache/kafka/common/internals/PartitionStates.java +++ b/clients/src/main/java/org/apache/kafka/common/internals/PartitionStates.java @@ -36,11 +36,17 @@ * topic would "wrap around" and appear twice. However, as partitions are fetched in different orders and partition * leadership changes, we will deviate from the optimal. If this turns out to be an issue in practice, we can improve * it by tracking the partitions per node or calling `set` every so often. + * + * Note that this class is not thread-safe with the exception of {@link #size()} which returns the number of + * partitions currently tracked. */ public class PartitionStates { private final LinkedHashMap map = new LinkedHashMap<>(); + /* the number of partitions that are currently assigned available in a thread safe manner */ + private volatile int size = 0; + public PartitionStates() {} public void moveToEnd(TopicPartition topicPartition) { @@ -52,10 +58,12 @@ public void moveToEnd(TopicPartition topicPartition) { public void updateAndMoveToEnd(TopicPartition topicPartition, S state) { map.remove(topicPartition); map.put(topicPartition, state); + updateSize(); } public void remove(TopicPartition topicPartition) { map.remove(topicPartition); + updateSize(); } /** @@ -67,6 +75,7 @@ public Set partitionSet() { public void clear() { map.clear(); + updateSize(); } public boolean contains(TopicPartition topicPartition) { @@ -95,8 +104,11 @@ public S stateValue(TopicPartition topicPartition) { return map.get(topicPartition); } + /** + * Get the number of partitions that are currently being tracked. This is thread-safe. + */ public int size() { - return map.size(); + return size; } /** @@ -108,6 +120,11 @@ public int size() { public void set(Map partitionToState) { map.clear(); update(partitionToState); + updateSize(); + } + + private void updateSize() { + size = map.size(); } private void update(Map partitionToState) { 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 a6da9f90397d2..9e2b6f18f0cfc 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 @@ -41,6 +41,8 @@ import java.util.concurrent.ThreadFactory; import java.util.concurrent.TimeUnit; +import static java.util.Collections.emptyList; + /** * A registry of sensors and metrics. *

@@ -446,6 +448,9 @@ public void removeSensor(String name) { removeMetric(metric.metricName()); log.debug("Removed sensor with name {}", name); childSensors = childrenSensors.remove(sensor); + for (final Sensor parent : sensor.parents()) { + childrenSensors.getOrDefault(parent, emptyList()).remove(sensor); + } } } } 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 e4bf1aeee699d..1af9419bc757a 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 @@ -22,7 +22,6 @@ import org.apache.kafka.common.utils.Utils; import java.util.ArrayList; -import java.util.Collections; import java.util.HashSet; import java.util.LinkedHashMap; import java.util.LinkedList; @@ -32,6 +31,9 @@ import java.util.Set; import java.util.concurrent.TimeUnit; +import static java.util.Arrays.asList; +import static java.util.Collections.unmodifiableList; + /** * A sensor applies a continuous sequence of numerical values to a set of associated metrics. For example a sensor on * message size would record a sequence of message sizes using the {@link #record(double)} api and would maintain a set @@ -48,6 +50,7 @@ public final class Sensor { private final Time time; private volatile long lastRecordTime; private final long inactiveSensorExpirationTimeMs; + private final Object metricLock; public enum RecordingLevel { INFO(0, "INFO"), DEBUG(1, "DEBUG"); @@ -113,6 +116,7 @@ public boolean shouldRecord(final int configId) { this.inactiveSensorExpirationTimeMs = TimeUnit.MILLISECONDS.convert(inactiveSensorExpirationTimeSeconds, TimeUnit.SECONDS); this.lastRecordTime = time.milliseconds(); this.recordingLevel = recordingLevel; + this.metricLock = new Object(); checkForest(new HashSet()); } @@ -131,6 +135,10 @@ public String name() { return this.name; } + List parents() { + return unmodifiableList(asList(parents)); + } + /** * Record an occurrence, this is just short-hand for {@link #record(double) record(1.0)} */ @@ -174,9 +182,11 @@ public void record(double value, long timeMs, boolean checkQuotas) { if (shouldRecord()) { this.lastRecordTime = timeMs; synchronized (this) { - // increment all the stats - for (Stat stat : this.stats) - stat.record(config, value, timeMs); + synchronized (metricLock()) { + // increment all the stats + for (Stat stat : this.stats) + stat.record(config, value, timeMs); + } if (checkQuotas) checkQuotas(timeMs); } @@ -229,7 +239,7 @@ public synchronized boolean add(CompoundStat stat, MetricConfig config) { return false; this.stats.add(Utils.notNull(stat)); - Object lock = metricLock(stat); + Object lock = metricLock(); for (NamedMeasurable m : stat.stats()) { final KafkaMetric metric = new KafkaMetric(lock, m.name(), m.stat(), config == null ? this.config : config, time); if (!metrics.containsKey(metric.metricName())) { @@ -265,7 +275,7 @@ public synchronized boolean add(final MetricName metricName, final MeasurableSta return true; } else { final KafkaMetric metric = new KafkaMetric( - metricLock(stat), + metricLock(), Utils.notNull(metricName), Utils.notNull(stat), config == null ? this.config : config, @@ -287,14 +297,30 @@ public boolean hasExpired() { } synchronized List metrics() { - return Collections.unmodifiableList(new LinkedList<>(this.metrics.values())); + return unmodifiableList(new LinkedList<>(this.metrics.values())); } /** - * KafkaMetrics of sensors which use SampledStat should be synchronized on the Sensor object - * to allow concurrent reads and updates. For simplicity, all sensors are synchronized on Sensor. + * KafkaMetrics of sensors which use SampledStat should be synchronized on the same lock + * for sensor record and metric value read to allow concurrent reads and updates. For simplicity, + * all sensors are synchronized on this object. + *

+ * Sensor object is not used as a lock for reading metric value since metrics reporter is + * invoked while holding Sensor and Metrics locks to report addition and removal of metrics + * and synchronized reporters may deadlock if Sensor lock is used for reading metrics values. + * Note that Sensor object itself is used as a lock to protect the access to stats and metrics + * while recording metric values, adding and deleting sensors. + *

+ * Locking order (assume all MetricsReporter methods may be synchronized): + *

    + *
  • Sensor#add: Sensor -> Metrics -> MetricsReporter
  • + *
  • Metrics#removeSensor: Sensor -> Metrics -> MetricsReporter
  • + *
  • KafkaMetric#metricValue: MetricsReporter -> Sensor#metricLock
  • + *
  • Sensor#record: Sensor -> Sensor#metricLock
  • + *
+ *

*/ - private Object metricLock(Stat stat) { - return this; + private Object metricLock() { + return metricLock; } } diff --git a/clients/src/main/java/org/apache/kafka/common/metrics/stats/Meter.java b/clients/src/main/java/org/apache/kafka/common/metrics/stats/Meter.java index 09263cecae89c..91d4461d2b52f 100644 --- a/clients/src/main/java/org/apache/kafka/common/metrics/stats/Meter.java +++ b/clients/src/main/java/org/apache/kafka/common/metrics/stats/Meter.java @@ -61,6 +61,9 @@ public Meter(SampledStat rateStat, MetricName rateMetricName, MetricName totalMe * Construct a Meter with provided time unit and provided {@link SampledStat} stats for Rate */ public Meter(TimeUnit unit, SampledStat rateStat, MetricName rateMetricName, MetricName totalMetricName) { + if (!(rateStat instanceof SampledTotal) && !(rateStat instanceof Count)) { + throw new IllegalArgumentException("Meter is supported only for SampledTotal and Count"); + } this.total = new Total(); this.rate = new Rate(unit, rateStat); this.rateMetricName = rateMetricName; @@ -77,6 +80,8 @@ public List stats() { @Override public void record(MetricConfig config, double value, long timeMs) { rate.record(config, value, timeMs); - total.record(config, value, timeMs); + // Total metrics with Count stat should record 1.0 (as recorded in the count) + double totalValue = (rate.stat instanceof Count) ? 1.0 : value; + total.record(config, totalValue, timeMs); } } 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 564fbcd8c217c..55354ac8d6417 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 @@ -19,7 +19,6 @@ import java.io.EOFException; import java.io.IOException; import java.nio.ByteBuffer; -import java.nio.channels.ReadableByteChannel; import java.nio.channels.ScatteringByteChannel; import org.apache.kafka.common.memory.MemoryPool; import org.slf4j.Logger; @@ -90,33 +89,6 @@ public boolean complete() { } public long readFrom(ScatteringByteChannel channel) throws IOException { - return readFromReadableChannel(channel); - } - - @Override - public boolean requiredMemoryAmountKnown() { - return requestedBufferSize != -1; - } - - @Override - public boolean memoryAllocated() { - return buffer != null; - } - - - @Override - public void close() throws IOException { - if (buffer != null && buffer != EMPTY_BUFFER) { - memoryPool.release(buffer); - buffer = null; - } - } - - // 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); @@ -151,8 +123,35 @@ public long readFromReadableChannel(ReadableByteChannel channel) throws IOExcept return read; } + @Override + public boolean requiredMemoryAmountKnown() { + return requestedBufferSize != -1; + } + + @Override + public boolean memoryAllocated() { + return buffer != null; + } + + + @Override + public void close() throws IOException { + if (buffer != null && buffer != EMPTY_BUFFER) { + memoryPool.release(buffer); + buffer = null; + } + } + public ByteBuffer payload() { return this.buffer; } + /** + * Returns the total size of the receive including payload and size buffer + * for use in metrics. This is consistent with {@link NetworkSend#size()} + */ + public int size() { + return payload().limit() + size.limit(); + } + } 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 334ca79f0350d..057b1ac928e03 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 @@ -31,6 +31,7 @@ import org.apache.kafka.common.metrics.stats.SampledStat; import org.apache.kafka.common.utils.LogContext; import org.apache.kafka.common.utils.Time; +import org.apache.kafka.common.utils.Utils; import org.slf4j.Logger; import java.io.IOException; @@ -53,6 +54,7 @@ import java.util.Map; import java.util.Set; import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicReference; /** * A nioSelector interface for doing non-blocking multi-connection network I/O. @@ -265,6 +267,7 @@ private void configureSocketChannel(SocketChannel socketChannel, int sendBufferS public void register(String id, SocketChannel socketChannel) throws IOException { ensureNotRegistered(id); registerChannel(id, socketChannel, SelectionKey.OP_READ); + this.sensors.connectionCreated.record(); } private void ensureNotRegistered(String id) { @@ -278,6 +281,8 @@ private SelectionKey registerChannel(String id, SocketChannel socketChannel, int SelectionKey key = socketChannel.register(nioSelector, interestedOps); KafkaChannel channel = buildAndAttachKafkaChannel(socketChannel, id, key); this.channels.put(id, channel); + if (idleExpiryManager != null) + idleExpiryManager.update(channel.id(), time.nanoseconds()); return key; } @@ -310,15 +315,24 @@ public void wakeup() { @Override public void close() { List connections = new ArrayList<>(channels.keySet()); - for (String id : connections) - close(id); try { - this.nioSelector.close(); - } catch (IOException | SecurityException e) { - log.error("Exception closing nioSelector:", e); + for (String id : connections) + close(id); + } finally { + // If there is any exception thrown in close(id), we should still be able + // to close the remaining objects, especially the sensors because keeping + // the sensors may lead to failure to start up the ReplicaFetcherThread if + // the old sensors with the same names has not yet been cleaned up. + AtomicReference firstException = new AtomicReference<>(); + Utils.closeQuietly(nioSelector, "nioSelector", firstException); + Utils.closeQuietly(sensors, "sensors", firstException); + Utils.closeQuietly(channelBuilder, "channelBuilder", firstException); + Throwable exception = firstException.get(); + if (exception instanceof RuntimeException && !(exception instanceof SecurityException)) { + throw (RuntimeException) exception; + } + } - sensors.close(); - channelBuilder.close(); } /** @@ -498,7 +512,9 @@ void pollSelectionKeys(Set selectionKeys, //this channel has bytes enqueued in intermediary buffers that we could not read //(possibly because no memory). it may be the case that the underlying socket will //not come up in the next poll() and so we need to remember this channel for the - //next poll call otherwise data may be stuck in said buffers forever. + //next poll call otherwise data may be stuck in said buffers forever. If we attempt + //to process buffered data and no progress is made, the channel buffered status is + //cleared to avoid the overhead of checking every time. keysWithBufferedRead.add(key); } @@ -859,7 +875,7 @@ private void addToCompletedReceives() { private void addToCompletedReceives(KafkaChannel channel, Deque stagedDeque) { NetworkReceive networkReceive = stagedDeque.poll(); this.completedReceives.add(networkReceive); - this.sensors.recordBytesReceived(channel.id(), networkReceive.payload().limit()); + this.sensors.recordBytesReceived(channel.id(), networkReceive.size()); } // only for testing @@ -873,7 +889,7 @@ public int numStagedReceives(KafkaChannel channel) { return deque == null ? 0 : deque.size(); } - private class SelectorMetrics { + private class SelectorMetrics implements AutoCloseable { private final Metrics metrics; private final String metricGrpPrefix; private final Map metricTags; @@ -1053,6 +1069,7 @@ public void recordBytesReceived(String connection, int bytes) { } } + @Override public void close() { for (MetricName metricName : topLevelMetricNames) metrics.removeMetric(metricName); 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 index 704a19818e2bb..5dc8086613966 100644 --- a/clients/src/main/java/org/apache/kafka/common/network/SslTransportLayer.java +++ b/clients/src/main/java/org/apache/kafka/common/network/SslTransportLayer.java @@ -31,8 +31,10 @@ import javax.net.ssl.SSLEngineResult.Status; import javax.net.ssl.SSLException; import javax.net.ssl.SSLHandshakeException; -import javax.net.ssl.SSLSession; +import javax.net.ssl.SSLKeyException; import javax.net.ssl.SSLPeerUnverifiedException; +import javax.net.ssl.SSLProtocolException; +import javax.net.ssl.SSLSession; import org.apache.kafka.common.errors.SslAuthenticationException; import org.apache.kafka.common.security.auth.KafkaPrincipal; @@ -45,6 +47,7 @@ */ public class SslTransportLayer implements TransportLayer { private enum State { + NOT_INITALIZED, HANDSHAKE, HANDSHAKE_FAILED, READY, @@ -64,12 +67,11 @@ private enum State { private ByteBuffer netReadBuffer; private ByteBuffer netWriteBuffer; private ByteBuffer appReadBuffer; + private boolean hasBytesBuffered; private ByteBuffer emptyBuf = ByteBuffer.allocate(0); public static SslTransportLayer create(String channelId, SelectionKey key, SSLEngine sslEngine) throws IOException { - SslTransportLayer transportLayer = new SslTransportLayer(channelId, key, sslEngine); - transportLayer.startHandshake(); - return transportLayer; + return new SslTransportLayer(channelId, key, sslEngine); } // Prefer `create`, only use this in tests @@ -78,6 +80,7 @@ public static SslTransportLayer create(String channelId, SelectionKey key, SSLEn this.key = key; this.socketChannel = (SocketChannel) key.channel(); this.sslEngine = sslEngine; + this.state = State.NOT_INITALIZED; final LogContext logContext = new LogContext(String.format("[SslTransportLayer channelId=%s key=%s] ", channelId, key)); this.log = logContext.logger(getClass()); @@ -85,7 +88,7 @@ public static SslTransportLayer create(String channelId, SelectionKey key, SSLEn // Visible for testing protected void startHandshake() throws IOException { - if (state != null) + if (state != State.NOT_INITALIZED) throw new IllegalStateException("startHandshake() can only be called once, state " + state); this.netReadBuffer = ByteBuffer.allocate(netReadBufferSize()); @@ -153,11 +156,12 @@ public boolean isConnected() { */ @Override public void close() throws IOException { + State prevState = state; if (state == State.CLOSING) return; state = State.CLOSING; sslEngine.closeOutbound(); try { - if (isConnected()) { + if (prevState != State.NOT_INITALIZED && isConnected()) { if (!flush(netWriteBuffer)) { throw new IOException("Remaining data in the network buffer, can't send SSL close message."); } @@ -174,10 +178,13 @@ public void close() throws IOException { flush(netWriteBuffer); } } catch (IOException ie) { - log.warn("Failed to send SSL Close message", ie); + log.debug("Failed to send SSL Close message", ie); } finally { socketChannel.socket().close(); socketChannel.close(); + netReadBuffer = null; + netWriteBuffer = null; + appReadBuffer = null; } } @@ -239,6 +246,8 @@ protected boolean flush(ByteBuffer buf) throws IOException { */ @Override public void handshake() throws IOException { + if (state == State.NOT_INITALIZED) + startHandshake(); if (state == State.READY) throw renegotiationException(); if (state == State.CLOSING) @@ -254,17 +263,17 @@ public void handshake() throws IOException { doHandshake(); } catch (SSLException e) { - handshakeFailure(e, true); + maybeProcessHandshakeFailure(e, true, null); } catch (IOException e) { maybeThrowSslAuthenticationException(); // this exception could be due to a write. If there is data available to unwrap, - // process the data so that any SSLExceptions are reported + // process the data so that any SSL handshake exceptions are reported if (handshakeStatus == HandshakeStatus.NEED_UNWRAP && netReadBuffer.position() > 0) { try { handshakeUnwrap(false); } catch (SSLException e1) { - handshakeFailure(e1, false); + maybeProcessHandshakeFailure(e1, false, e); } } // If we get here, this is not a handshake failure, throw the original IOException @@ -503,13 +512,17 @@ public int read(ByteBuffer dst) throws IOException { read = readFromAppBuffer(dst); } + boolean readFromNetwork = false; boolean isClosed = false; // Each loop reads at most once from the socket. while (dst.remaining() > 0) { int netread = 0; netReadBuffer = Utils.ensureCapacity(netReadBuffer, netReadBufferSize()); - if (netReadBuffer.remaining() > 0) + if (netReadBuffer.remaining() > 0) { netread = readFromSocketChannel(); + if (netread > 0) + readFromNetwork = true; + } while (netReadBuffer.position() > 0) { netReadBuffer.flip(); @@ -563,6 +576,7 @@ public int read(ByteBuffer dst) throws IOException { if (netread <= 0 || isClosed) break; } + updateBytesBuffered(readFromNetwork || read > 0); // If data has been read and unwrapped, return the data even if end-of-stream, channel will be closed // on a subsequent poll. return read; @@ -793,6 +807,11 @@ protected ByteBuffer netReadBuffer() { return netReadBuffer; } + // Visibility for testing + protected ByteBuffer appReadBuffer() { + return appReadBuffer; + } + /** * SSL exceptions are propagated as authentication failures so that clients can avoid * retries and report the failure. If `flush` is true, exceptions are propagated after @@ -813,6 +832,32 @@ private void handshakeFailure(SSLException sslException, boolean flush) throws I throw handshakeException; } + // SSL handshake failures are typically thrown as SSLHandshakeException, SSLProtocolException, + // SSLPeerUnverifiedException or SSLKeyException if the cause is known. These exceptions indicate + // authentication failures (e.g. configuration errors) which should not be retried. But the SSL engine + // may also throw exceptions using the base class SSLException in a few cases: + // a) If there are no matching ciphers or TLS version or the private key is invalid, client will be + // unable to process the server message and an SSLException is thrown: + // javax.net.ssl.SSLException: Unrecognized SSL message, plaintext connection? + // b) If server closes the connection gracefully during handshake, client may receive close_notify + // and and an SSLException is thrown: + // javax.net.ssl.SSLException: Received close_notify during handshake + // We want to handle a) as a non-retriable SslAuthenticationException and b) as a retriable IOException. + // To do this we need to rely on the exception string. Since it is safer to throw a retriable exception + // when we are not sure, we will treat only the first exception string as a handshake exception. + private void maybeProcessHandshakeFailure(SSLException sslException, boolean flush, IOException ioException) throws IOException { + if (sslException instanceof SSLHandshakeException || sslException instanceof SSLProtocolException || + sslException instanceof SSLPeerUnverifiedException || sslException instanceof SSLKeyException || + sslException.getMessage().contains("Unrecognized SSL message")) + handshakeFailure(sslException, flush); + else if (ioException == null) + throw sslException; + else { + log.debug("SSLException while unwrapping data after IOException, original IOException will be propagated", sslException); + throw ioException; + } + } + // If handshake has already failed, throw the authentication exception. private void maybeThrowSslAuthenticationException() { if (handshakeException != null) @@ -826,12 +871,22 @@ public boolean isMute() { @Override public boolean hasBytesBuffered() { - return netReadBuffer.position() != 0 || appReadBuffer.position() != 0; + return hasBytesBuffered; + } + + // Update `hasBytesBuffered` status. If any bytes were read from the network or + // if data was returned from read, `hasBytesBuffered` is set to true if any buffered + // data is still remaining. If not, `hasBytesBuffered` is set to false since no progress + // can be made until more data is available to read from the network. + private void updateBytesBuffered(boolean madeProgress) { + if (madeProgress) + hasBytesBuffered = netReadBuffer.position() != 0 || appReadBuffer.position() != 0; + else + hasBytesBuffered = false; } @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/TransportLayer.java b/clients/src/main/java/org/apache/kafka/common/network/TransportLayer.java index 3673d21dae6b5..a8a4b8730283c 100644 --- a/clients/src/main/java/org/apache/kafka/common/network/TransportLayer.java +++ b/clients/src/main/java/org/apache/kafka/common/network/TransportLayer.java @@ -94,6 +94,7 @@ public interface TransportLayer extends ScatteringByteChannel, GatheringByteChan /** * @return true if channel has bytes to be read in any intermediate buffers + * which may be processed without reading additional data from the network. */ boolean hasBytesBuffered(); diff --git a/clients/src/main/java/org/apache/kafka/common/protocol/CommonFields.java b/clients/src/main/java/org/apache/kafka/common/protocol/CommonFields.java index 96fa136011d7a..9eddf2b17e668 100644 --- a/clients/src/main/java/org/apache/kafka/common/protocol/CommonFields.java +++ b/clients/src/main/java/org/apache/kafka/common/protocol/CommonFields.java @@ -17,7 +17,7 @@ package org.apache.kafka.common.protocol; import org.apache.kafka.common.protocol.types.Field; -import org.apache.kafka.common.resource.ResourceNameType; +import org.apache.kafka.common.resource.PatternType; public class CommonFields { public static final Field.Int32 THROTTLE_TIME_MS = new Field.Int32("throttle_time_ms", @@ -46,8 +46,8 @@ public class CommonFields { public static final Field.Int8 RESOURCE_TYPE = new Field.Int8("resource_type", "The resource type"); public static final Field.Str RESOURCE_NAME = new Field.Str("resource_name", "The resource name"); public static final Field.NullableStr RESOURCE_NAME_FILTER = new Field.NullableStr("resource_name", "The resource name filter"); - public static final Field.Int8 RESOURCE_NAME_TYPE = new Field.Int8("resource_name_type", "The resource name type", ResourceNameType.LITERAL.code()); - public static final Field.Int8 RESOURCE_NAME_TYPE_FILTER = new Field.Int8("resource_name_type_filter", "The resource name type filter", ResourceNameType.LITERAL.code()); + public static final Field.Int8 RESOURCE_PATTERN_TYPE = new Field.Int8("resource_pattten_type", "The resource pattern type", PatternType.LITERAL.code()); + public static final Field.Int8 RESOURCE_PATTERN_TYPE_FILTER = new Field.Int8("resource_pattern_type_filter", "The resource pattern type filter", PatternType.LITERAL.code()); public static final Field.Str PRINCIPAL = new Field.Str("principal", "The ACL principal"); public static final Field.NullableStr PRINCIPAL_FILTER = new Field.NullableStr("principal", "The ACL principal filter"); public static final Field.Str HOST = new Field.Str("host", "The ACL host"); 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 5db1d314be301..9c522dff935ec 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 @@ -30,6 +30,7 @@ import org.apache.kafka.common.errors.DelegationTokenExpiredException; import org.apache.kafka.common.errors.DelegationTokenNotFoundException; import org.apache.kafka.common.errors.DelegationTokenOwnerMismatchException; +import org.apache.kafka.common.errors.ListenerNotFoundException; import org.apache.kafka.common.errors.FetchSessionIdNotFoundException; import org.apache.kafka.common.errors.GroupAuthorizationException; import org.apache.kafka.common.errors.GroupIdNotFoundException; @@ -624,7 +625,14 @@ public ApiException build(String message) { public ApiException build(String message) { return new InvalidFetchSessionEpochException(message); } - }); + }), + LISTENER_NOT_FOUND(72, "There is no listener on the leader broker that matches the listener on which metadata request was processed", + new ApiExceptionBuilder() { + @Override + public ApiException build(String message) { + return new ListenerNotFoundException(message); + } + }),; private interface ApiExceptionBuilder { ApiException build(String message); 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 7dccc1015aad0..7183aedbd951a 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 @@ -111,6 +111,12 @@ public Short getOrElse(Field.Int16 field, short alternative) { return alternative; } + public Byte getOrElse(Field.Int8 field, byte alternative) { + if (hasField(field.name)) + return getByte(field.name); + return alternative; + } + public Integer getOrElse(Field.Int32 field, int alternative) { if (hasField(field.name)) return getInt(field.name); diff --git a/clients/src/main/java/org/apache/kafka/common/record/AbstractLegacyRecordBatch.java b/clients/src/main/java/org/apache/kafka/common/record/AbstractLegacyRecordBatch.java index 6ac073dd29801..291873e81cc09 100644 --- a/clients/src/main/java/org/apache/kafka/common/record/AbstractLegacyRecordBatch.java +++ b/clients/src/main/java/org/apache/kafka/common/record/AbstractLegacyRecordBatch.java @@ -29,7 +29,6 @@ import java.io.IOException; import java.io.InputStream; import java.nio.ByteBuffer; -import java.nio.channels.FileChannel; import java.util.ArrayDeque; import java.util.Iterator; import java.util.NoSuchElementException; @@ -528,10 +527,10 @@ static class LegacyFileChannelRecordBatch extends FileLogInputStream.FileChannel LegacyFileChannelRecordBatch(long offset, byte magic, - FileChannel channel, + FileRecords fileRecords, int position, int batchSize) { - super(offset, magic, channel, position, batchSize); + super(offset, magic, fileRecords, position, batchSize); } @Override diff --git a/clients/src/main/java/org/apache/kafka/common/record/BufferSupplier.java b/clients/src/main/java/org/apache/kafka/common/record/BufferSupplier.java index 2e09f7d1a2cc3..1a6c92c712fba 100644 --- a/clients/src/main/java/org/apache/kafka/common/record/BufferSupplier.java +++ b/clients/src/main/java/org/apache/kafka/common/record/BufferSupplier.java @@ -93,4 +93,36 @@ public void close() { bufferMap.clear(); } } + + /** + * Simple buffer supplier for single-threaded usage. It caches a single buffer, which grows + * monotonically as needed to fulfill the allocation request. + */ + public static class GrowableBufferSupplier extends BufferSupplier { + private ByteBuffer cachedBuffer; + + @Override + public ByteBuffer get(int minCapacity) { + if (cachedBuffer != null && cachedBuffer.capacity() >= minCapacity) { + ByteBuffer res = cachedBuffer; + cachedBuffer = null; + return res; + } else { + cachedBuffer = null; + return ByteBuffer.allocate(minCapacity); + } + } + + @Override + public void release(ByteBuffer buffer) { + buffer.clear(); + cachedBuffer = buffer; + } + + @Override + public void close() { + cachedBuffer = null; + } + } + } diff --git a/clients/src/main/java/org/apache/kafka/common/record/DefaultRecordBatch.java b/clients/src/main/java/org/apache/kafka/common/record/DefaultRecordBatch.java index 71e668e45da01..19ddb0ef3fa1a 100644 --- a/clients/src/main/java/org/apache/kafka/common/record/DefaultRecordBatch.java +++ b/clients/src/main/java/org/apache/kafka/common/record/DefaultRecordBatch.java @@ -28,7 +28,6 @@ import java.io.IOException; import java.nio.BufferUnderflowException; import java.nio.ByteBuffer; -import java.nio.channels.FileChannel; import java.util.ArrayList; import java.util.Collections; import java.util.Iterator; @@ -523,12 +522,18 @@ static int estimateBatchSizeUpperBound(ByteBuffer key, ByteBuffer value, Header[ return RECORD_BATCH_OVERHEAD + DefaultRecord.recordSizeUpperBound(key, value, headers); } - static int incrementSequence(int baseSequence, int increment) { + public static int incrementSequence(int baseSequence, int increment) { if (baseSequence > Integer.MAX_VALUE - increment) return increment - (Integer.MAX_VALUE - baseSequence) - 1; return baseSequence + increment; } + public static int decrementSequence(int baseSequence, int decrement) { + if (baseSequence < decrement) + return Integer.MAX_VALUE - (decrement - baseSequence) + 1; + return baseSequence - decrement; + } + private abstract class RecordIterator implements CloseableIterator { private final Long logAppendTime; private final long baseOffset; @@ -586,10 +591,10 @@ static class DefaultFileChannelRecordBatch extends FileLogInputStream.FileChanne DefaultFileChannelRecordBatch(long offset, byte magic, - FileChannel channel, + FileRecords fileRecords, int position, int batchSize) { - super(offset, magic, channel, position, batchSize); + super(offset, magic, fileRecords, position, batchSize); } @Override diff --git a/clients/src/main/java/org/apache/kafka/common/record/FileLogInputStream.java b/clients/src/main/java/org/apache/kafka/common/record/FileLogInputStream.java index 92e8864a183b6..472c7a7ac3e13 100644 --- a/clients/src/main/java/org/apache/kafka/common/record/FileLogInputStream.java +++ b/clients/src/main/java/org/apache/kafka/common/record/FileLogInputStream.java @@ -82,9 +82,9 @@ public FileChannelRecordBatch nextBatch() throws IOException { final FileChannelRecordBatch batch; if (magic < RecordBatch.MAGIC_VALUE_V2) - batch = new LegacyFileChannelRecordBatch(offset, magic, channel, position, size); + batch = new LegacyFileChannelRecordBatch(offset, magic, fileRecords, position, size); else - batch = new DefaultFileChannelRecordBatch(offset, magic, channel, position, size); + batch = new DefaultFileChannelRecordBatch(offset, magic, fileRecords, position, size); position += batch.sizeInBytes(); return batch; @@ -98,7 +98,7 @@ public FileChannelRecordBatch nextBatch() throws IOException { public abstract static class FileChannelRecordBatch extends AbstractRecordBatch { protected final long offset; protected final byte magic; - protected final FileChannel channel; + protected final FileRecords fileRecords; protected final int position; protected final int batchSize; @@ -107,12 +107,12 @@ public abstract static class FileChannelRecordBatch extends AbstractRecordBatch FileChannelRecordBatch(long offset, byte magic, - FileChannel channel, + FileRecords fileRecords, int position, int batchSize) { this.offset = offset; this.magic = magic; - this.channel = channel; + this.fileRecords = fileRecords; this.position = position; this.batchSize = batchSize; } @@ -173,14 +173,14 @@ public int sizeInBytes() { @Override public void writeTo(ByteBuffer buffer) { + FileChannel channel = fileRecords.channel(); try { int limit = buffer.limit(); buffer.limit(buffer.position() + sizeInBytes()); Utils.readFully(channel, buffer, position); buffer.limit(limit); } catch (IOException e) { - throw new KafkaException("Failed to read record batch at position " + position + " from file channel " + - channel, e); + throw new KafkaException("Failed to read record batch at position " + position + " from " + fileRecords, e); } } @@ -207,13 +207,14 @@ protected RecordBatch loadBatchHeader() { } private RecordBatch loadBatchWithSize(int size, String description) { + FileChannel channel = fileRecords.channel(); try { ByteBuffer buffer = ByteBuffer.allocate(size); Utils.readFullyOrFail(channel, buffer, position, description); buffer.rewind(); return toMemoryRecordBatch(buffer); } catch (IOException e) { - throw new KafkaException(e); + throw new KafkaException("Failed to load record batch at position " + position + " from " + fileRecords, e); } } @@ -226,14 +227,19 @@ public boolean equals(Object o) { FileChannelRecordBatch that = (FileChannelRecordBatch) o; + FileChannel channel = fileRecords == null ? null : fileRecords.channel(); + FileChannel thatChannel = that.fileRecords == null ? null : that.fileRecords.channel(); + return offset == that.offset && position == that.position && batchSize == that.batchSize && - (channel == null ? that.channel == null : channel.equals(that.channel)); + (channel == null ? thatChannel == null : channel.equals(thatChannel)); } @Override public int hashCode() { + FileChannel channel = fileRecords == null ? null : fileRecords.channel(); + int result = (int) (offset ^ (offset >>> 32)); result = 31 * result + (channel != null ? channel.hashCode() : 0); result = 31 * result + position; diff --git a/clients/src/main/java/org/apache/kafka/common/record/FileRecords.java b/clients/src/main/java/org/apache/kafka/common/record/FileRecords.java index e44d5d93be814..df38ac78ba37e 100644 --- a/clients/src/main/java/org/apache/kafka/common/record/FileRecords.java +++ b/clients/src/main/java/org/apache/kafka/common/record/FileRecords.java @@ -109,14 +109,12 @@ public FileChannel channel() { * * @param buffer The buffer to write the batches to * @param position Position in the buffer to read from - * @return The same buffer * @throws IOException If an I/O error occurs, see {@link FileChannel#read(ByteBuffer, long)} for details on the * possible exceptions */ - public ByteBuffer readInto(ByteBuffer buffer, int position) throws IOException { + public void readInto(ByteBuffer buffer, int position) throws IOException { Utils.readFully(channel, buffer, position + this.start); buffer.flip(); - return buffer; } /** @@ -131,7 +129,7 @@ public ByteBuffer readInto(ByteBuffer buffer, int position) throws IOException { * @param size The number of bytes after the start position to include * @return A sliced wrapper on this message set limited based on the given position and size */ - public FileRecords read(int position, int size) throws IOException { + public FileRecords slice(int position, int size) throws IOException { if (position < 0) throw new IllegalArgumentException("Invalid position: " + position + " in read from " + file); if (size < 0) @@ -356,7 +354,14 @@ public String toString() { ")"; } - private Iterable batchesFrom(final int start) { + /** + * Get an iterator over the record batches in the file, starting at a specific position. This is similar to + * {@link #batches()} except that callers specify a particular position to start reading the batches from. This + * method must be used with caution: the start position passed in must be a known start of a batch. + * @param start The position to start record iteration from; must be a known position for start of a batch + * @return An iterator over batches starting from {@code start} + */ + public Iterable batchesFrom(final int start) { return new Iterable() { @Override public Iterator iterator() { diff --git a/clients/src/main/java/org/apache/kafka/common/record/LazyDownConversionRecords.java b/clients/src/main/java/org/apache/kafka/common/record/LazyDownConversionRecords.java index da14b5b494f2d..d58689de119a8 100644 --- a/clients/src/main/java/org/apache/kafka/common/record/LazyDownConversionRecords.java +++ b/clients/src/main/java/org/apache/kafka/common/record/LazyDownConversionRecords.java @@ -21,7 +21,6 @@ import org.apache.kafka.common.utils.Time; import java.util.ArrayList; -import java.util.Collections; import java.util.List; import java.util.Objects; @@ -57,13 +56,15 @@ public LazyDownConversionRecords(TopicPartition topicPartition, Records records, // need to make sure that we are able to accommodate one full batch of down-converted messages. The way we achieve // this is by having sizeInBytes method factor in the size of the first down-converted batch and return at least // its size. - AbstractIterator it = records.batchIterator(); + java.util.Iterator it = iterator(0); if (it.hasNext()) { - firstConvertedBatch = RecordsUtil.downConvert(Collections.singletonList(it.peek()), toMagic, firstOffset, time); + firstConvertedBatch = it.next(); sizeInBytes = Math.max(records.sizeInBytes(), firstConvertedBatch.records().sizeInBytes()); } else { + // If there are no messages we got after down-conversion, make sure we are able to send at least an overflow + // message to the consumer. Typically, the consumer would need to increase the fetch size in such cases. firstConvertedBatch = null; - sizeInBytes = 0; + sizeInBytes = LazyDownConversionRecordsSend.MIN_OVERFLOW_MESSAGE_LENGTH; } } @@ -148,21 +149,28 @@ protected ConvertedRecords makeNext() { return convertedBatch; } - if (!batchIterator.hasNext()) - return allDone(); - - // Figure out batches we should down-convert based on the size constraints - List batches = new ArrayList<>(); - boolean isFirstBatch = true; - long sizeSoFar = 0; - while (batchIterator.hasNext() && - (isFirstBatch || (batchIterator.peek().sizeInBytes() + sizeSoFar) <= maximumReadSize)) { - RecordBatch currentBatch = batchIterator.next(); - batches.add(currentBatch); - sizeSoFar += currentBatch.sizeInBytes(); - isFirstBatch = false; + while (batchIterator.hasNext()) { + List batches = new ArrayList<>(); + boolean isFirstBatch = true; + long sizeSoFar = 0; + + // Figure out batches we should down-convert based on the size constraints + while (batchIterator.hasNext() && + (isFirstBatch || (batchIterator.peek().sizeInBytes() + sizeSoFar) <= maximumReadSize)) { + RecordBatch currentBatch = batchIterator.next(); + batches.add(currentBatch); + sizeSoFar += currentBatch.sizeInBytes(); + isFirstBatch = false; + } + ConvertedRecords convertedRecords = RecordsUtil.downConvert(batches, toMagic, firstOffset, time); + // During conversion, it is possible that we drop certain batches because they do not have an equivalent + // representation in the message format we want to convert to. For example, V0 and V1 message formats + // have no notion of transaction markers which were introduced in V2 so they get dropped during conversion. + // We return converted records only when we have at least one valid batch of messages after conversion. + if (convertedRecords.records().sizeInBytes() > 0) + return convertedRecords; } - return RecordsUtil.downConvert(batches, toMagic, firstOffset, time); + return allDone(); } } } diff --git a/clients/src/main/java/org/apache/kafka/common/record/LazyDownConversionRecordsSend.java b/clients/src/main/java/org/apache/kafka/common/record/LazyDownConversionRecordsSend.java index b78211418acfe..f0fab7d876d92 100644 --- a/clients/src/main/java/org/apache/kafka/common/record/LazyDownConversionRecordsSend.java +++ b/clients/src/main/java/org/apache/kafka/common/record/LazyDownConversionRecordsSend.java @@ -20,7 +20,6 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import java.io.EOFException; import java.io.IOException; import java.nio.ByteBuffer; import java.nio.channels.GatheringByteChannel; @@ -33,6 +32,7 @@ public final class LazyDownConversionRecordsSend extends RecordsSend { private static final Logger log = LoggerFactory.getLogger(LazyDownConversionRecordsSend.class); private static final int MAX_READ_SIZE = 128 * 1024; + static final int MIN_OVERFLOW_MESSAGE_LENGTH = Records.LOG_OVERHEAD; private RecordConversionStats recordConversionStats; private RecordsSend convertedRecordsWriter; @@ -49,41 +49,31 @@ public LazyDownConversionRecordsSend(String destination, LazyDownConversionRecor public long writeTo(GatheringByteChannel channel, long previouslyWritten, int remaining) throws IOException { if (convertedRecordsWriter == null || convertedRecordsWriter.completed()) { MemoryRecords convertedRecords; - // Check if we have more chunks left to down-convert if (convertedRecordsIterator.hasNext()) { // Get next chunk of down-converted messages ConvertedRecords recordsAndStats = convertedRecordsIterator.next(); convertedRecords = recordsAndStats.records(); - - int sizeOfFirstConvertedBatch = convertedRecords.batchIterator().next().sizeInBytes(); - if (previouslyWritten == 0 && sizeOfFirstConvertedBatch > size()) - throw new EOFException("Unable to send first batch completely." + - " maximum_size: " + size() + - " converted_records_size: " + sizeOfFirstConvertedBatch); - recordConversionStats.add(recordsAndStats.recordConversionStats()); - log.debug("Got lazy converted records for {" + topicPartition() + "} with length=" + convertedRecords.sizeInBytes()); + log.debug("Down-converted records for partition {} with length={}", topicPartition(), convertedRecords.sizeInBytes()); } else { - if (previouslyWritten == 0) - throw new EOFException("Unable to get the first batch of down-converted records"); - - // We do not have any records left to down-convert. Construct a "fake" message for the length remaining. + // We do not have any records left to down-convert. Construct an overflow message for the length remaining. // This message will be ignored by the consumer because its length will be past the length of maximum // possible response size. // DefaultRecordBatch => // BaseOffset => Int64 // Length => Int32 // ... - // TODO: check if there is a better way to encapsulate this logic, perhaps in DefaultRecordBatch - log.debug("Constructing fake message batch for topic-partition {" + topicPartition() + "} for remaining length " + remaining); - int minLength = (Long.SIZE / Byte.SIZE) + (Integer.SIZE / Byte.SIZE); - ByteBuffer fakeMessageBatch = ByteBuffer.allocate(Math.max(minLength, Math.min(remaining + 1, MAX_READ_SIZE))); - fakeMessageBatch.putLong(-1L); - fakeMessageBatch.putInt(remaining + 1); - convertedRecords = MemoryRecords.readableRecords(fakeMessageBatch); - } + ByteBuffer overflowMessageBatch = ByteBuffer.allocate( + Math.max(MIN_OVERFLOW_MESSAGE_LENGTH, Math.min(remaining + 1, MAX_READ_SIZE))); + overflowMessageBatch.putLong(-1L); + // Fill in the length of the overflow batch. A valid batch must be at least as long as the minimum batch + // overhead. + overflowMessageBatch.putInt(Math.max(remaining + 1, DefaultRecordBatch.RECORD_BATCH_OVERHEAD)); + convertedRecords = MemoryRecords.readableRecords(overflowMessageBatch); + log.debug("Constructed overflow message batch for partition {} with length={}", topicPartition(), remaining); + } convertedRecordsWriter = new DefaultRecordsSend(destination(), convertedRecords, Math.min(convertedRecords.sizeInBytes(), remaining)); } return convertedRecordsWriter.writeTo(channel); 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 55a471149c6ca..af62e09c57ecd 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 @@ -160,20 +160,14 @@ public FilterResult filterTo(TopicPartition partition, RecordFilter filter, Byte private static FilterResult filterTo(TopicPartition partition, Iterable batches, RecordFilter filter, ByteBuffer destinationBuffer, int maxRecordBatchSize, BufferSupplier decompressionBufferSupplier) { - long maxTimestamp = RecordBatch.NO_TIMESTAMP; - long maxOffset = -1L; - long shallowOffsetOfMaxTimestamp = -1L; - int messagesRead = 0; - int bytesRead = 0; // bytes processed from `batches` - int messagesRetained = 0; - int bytesRetained = 0; - + FilterResult filterResult = new FilterResult(destinationBuffer); ByteBufferOutputStream bufferOutputStream = new ByteBufferOutputStream(destinationBuffer); for (MutableRecordBatch batch : batches) { - bytesRead += batch.sizeInBytes(); - + long maxOffset = -1L; BatchRetention batchRetention = filter.checkBatchRetention(batch); + filterResult.bytesRead += batch.sizeInBytes(); + if (batchRetention == BatchRetention.DELETE) continue; @@ -189,7 +183,7 @@ private static FilterResult filterTo(TopicPartition partition, Iterable iterator = batch.streamingIterator(decompressionBufferSupplier)) { while (iterator.hasNext()) { Record record = iterator.next(); - messagesRead += 1; + filterResult.messagesRead += 1; if (filter.shouldRetainRecord(batch, record)) { // Check for log corruption due to KAFKA-4298. If we find it, make sure that we overwrite @@ -210,20 +204,11 @@ private static FilterResult filterTo(TopicPartition partition, Iterable maxTimestamp) { - maxTimestamp = batch.maxTimestamp(); - shallowOffsetOfMaxTimestamp = batch.lastOffset(); - } + filterResult.updateRetainedBatchMetadata(batch, retainedRecords.size(), false); } else { MemoryRecordsBuilder builder = buildRetainedRecordsInto(batch, retainedRecords, bufferOutputStream); MemoryRecords records = builder.build(); int filteredBatchSize = records.sizeInBytes(); - - messagesRetained += retainedRecords.size(); - bytesRetained += filteredBatchSize; - if (filteredBatchSize > batch.sizeInBytes() && filteredBatchSize > maxRecordBatchSize) log.warn("Record batch from {} with last offset {} exceeded max record batch size {} after cleaning " + "(new size is {}). Consumers with version earlier than 0.10.1.0 may need to " + @@ -231,10 +216,8 @@ private static FilterResult filterTo(TopicPartition partition, Iterable maxTimestamp) { - maxTimestamp = info.maxTimestamp; - shallowOffsetOfMaxTimestamp = info.shallowOffsetOfMaxTimestamp; - } + filterResult.updateRetainedBatchMetadata(info.maxTimestamp, info.shallowOffsetOfMaxTimestamp, + maxOffset, retainedRecords.size(), filteredBatchSize); } } else if (batchRetention == BatchRetention.RETAIN_EMPTY) { if (batchMagic < RecordBatch.MAGIC_VALUE_V2) @@ -245,18 +228,19 @@ private static FilterResult filterTo(TopicPartition partition, Iterable this.maxTimestamp) { + this.maxTimestamp = maxTimestamp; + this.shallowOffsetOfMaxTimestamp = shallowOffsetOfMaxTimestamp; + } + this.maxOffset = Math.max(maxOffset, this.maxOffset); + this.messagesRetained += messagesRetained; + this.bytesRetained += bytesRetained; + } + + private void validateBatchMetadata(long maxTimestamp, long shallowOffsetOfMaxTimestamp, long maxOffset) { + if (maxTimestamp != RecordBatch.NO_TIMESTAMP && shallowOffsetOfMaxTimestamp < 0) + throw new IllegalArgumentException("shallowOffset undefined for maximum timestamp " + maxTimestamp); + if (maxOffset < 0) + throw new IllegalArgumentException("maxOffset undefined"); + } + + public ByteBuffer outputBuffer() { + return outputBuffer; + } + + public int messagesRead() { + return messagesRead; + } + + public int bytesRead() { + return bytesRead; + } + + public int messagesRetained() { + return messagesRetained; + } + + public int bytesRetained() { + return bytesRetained; + } + + public long maxOffset() { + return maxOffset; + } + + public long maxTimestamp() { + return maxTimestamp; + } + + public long shallowOffsetOfMaxTimestamp() { + return shallowOffsetOfMaxTimestamp; } } diff --git a/clients/src/main/java/org/apache/kafka/common/record/RecordVersion.java b/clients/src/main/java/org/apache/kafka/common/record/RecordVersion.java index 1f80d623ca9fd..8406d5331c8a8 100644 --- a/clients/src/main/java/org/apache/kafka/common/record/RecordVersion.java +++ b/clients/src/main/java/org/apache/kafka/common/record/RecordVersion.java @@ -34,6 +34,15 @@ public enum RecordVersion { this.value = (byte) value; } + /** + * Check whether this version precedes another version. + * + * @return true only if the magic value is less than the other's + */ + public boolean precedes(RecordVersion other) { + return this.value < other.value; + } + public static RecordVersion lookup(byte value) { if (value < 0 || value >= VALUES.length) throw new IllegalArgumentException("Unknown record version: " + value); diff --git a/clients/src/main/java/org/apache/kafka/common/record/RecordsUtil.java b/clients/src/main/java/org/apache/kafka/common/record/RecordsUtil.java index c9b739413175c..291f241152188 100644 --- a/clients/src/main/java/org/apache/kafka/common/record/RecordsUtil.java +++ b/clients/src/main/java/org/apache/kafka/common/record/RecordsUtil.java @@ -17,6 +17,7 @@ package org.apache.kafka.common.record; import org.apache.kafka.common.utils.Time; +import org.apache.kafka.common.utils.Utils; import java.nio.ByteBuffer; import java.util.ArrayList; @@ -73,9 +74,11 @@ protected static ConvertedRecords downConvert(Iterable entries; public Config(Collection entries) { - this.entries = entries; + this.entries = Objects.requireNonNull(entries, "entries"); } public Collection entries() { @@ -86,8 +88,8 @@ public static class ConfigEntry { private final String value; public ConfigEntry(String name, String value) { - this.name = name; - this.value = value; + this.name = Objects.requireNonNull(name, "name"); + this.value = Objects.requireNonNull(value, "value"); } public String name() { @@ -102,12 +104,12 @@ public String value() { public static class Builder extends AbstractRequest.Builder { - private final Map configs; + private final Map configs; private final boolean validateOnly; - public Builder(Map configs, boolean validateOnly) { + public Builder(Map configs, boolean validateOnly) { super(ApiKeys.ALTER_CONFIGS); - this.configs = configs; + this.configs = Objects.requireNonNull(configs, "configs"); this.validateOnly = validateOnly; } @@ -117,12 +119,12 @@ public AlterConfigsRequest build(short version) { } } - private final Map configs; + private final Map configs; private final boolean validateOnly; - public AlterConfigsRequest(short version, Map configs, boolean validateOnly) { + public AlterConfigsRequest(short version, Map configs, boolean validateOnly) { super(version); - this.configs = configs; + this.configs = Objects.requireNonNull(configs, "configs"); this.validateOnly = validateOnly; } @@ -134,9 +136,9 @@ public AlterConfigsRequest(Struct struct, short version) { for (Object resourcesObj : resourcesArray) { Struct resourcesStruct = (Struct) resourcesObj; - ResourceType resourceType = ResourceType.forId(resourcesStruct.getByte(RESOURCE_TYPE_KEY_NAME)); + ConfigResource.Type resourceType = ConfigResource.Type.forId(resourcesStruct.getByte(RESOURCE_TYPE_KEY_NAME)); String resourceName = resourcesStruct.getString(RESOURCE_NAME_KEY_NAME); - Resource resource = new Resource(resourceType, resourceName); + ConfigResource resource = new ConfigResource(resourceType, resourceName); Object[] configEntriesArray = resourcesStruct.getArray(CONFIG_ENTRIES_KEY_NAME); List configEntries = new ArrayList<>(configEntriesArray.length); @@ -151,7 +153,7 @@ public AlterConfigsRequest(Struct struct, short version) { } } - public Map configs() { + public Map configs() { return configs; } @@ -164,10 +166,10 @@ protected Struct toStruct() { Struct struct = new Struct(ApiKeys.ALTER_CONFIGS.requestSchema(version())); struct.set(VALIDATE_ONLY_KEY_NAME, validateOnly); List resourceStructs = new ArrayList<>(configs.size()); - for (Map.Entry entry : configs.entrySet()) { + for (Map.Entry entry : configs.entrySet()) { Struct resourceStruct = struct.instance(RESOURCES_KEY_NAME); - Resource resource = entry.getKey(); + ConfigResource resource = entry.getKey(); resourceStruct.set(RESOURCE_TYPE_KEY_NAME, resource.type().id()); resourceStruct.set(RESOURCE_NAME_KEY_NAME, resource.name()); @@ -194,8 +196,8 @@ public AbstractResponse getErrorResponse(int throttleTimeMs, Throwable e) { case 0: case 1: ApiError error = ApiError.fromThrowable(e); - Map errors = new HashMap<>(configs.size()); - for (Resource resource : configs.keySet()) + Map errors = new HashMap<>(configs.size()); + for (ConfigResource resource : configs.keySet()) errors.put(resource, error); return new AlterConfigsResponse(throttleTimeMs, errors); default: @@ -207,5 +209,4 @@ public AbstractResponse getErrorResponse(int throttleTimeMs, Throwable e) { public static AlterConfigsRequest parse(ByteBuffer buffer, short version) { return new AlterConfigsRequest(ApiKeys.ALTER_CONFIGS.parseRequest(version, buffer), version); } - } diff --git a/clients/src/main/java/org/apache/kafka/common/requests/AlterConfigsResponse.java b/clients/src/main/java/org/apache/kafka/common/requests/AlterConfigsResponse.java index feb694bf077f5..bf805df5158e9 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/AlterConfigsResponse.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/AlterConfigsResponse.java @@ -17,11 +17,12 @@ package org.apache.kafka.common.requests; +import org.apache.kafka.common.config.ConfigResource; import org.apache.kafka.common.protocol.ApiKeys; +import org.apache.kafka.common.protocol.Errors; 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.Errors; import org.apache.kafka.common.protocol.types.Struct; import java.nio.ByteBuffer; @@ -29,6 +30,7 @@ import java.util.HashMap; import java.util.List; import java.util.Map; +import java.util.Objects; import static org.apache.kafka.common.protocol.CommonFields.ERROR_CODE; import static org.apache.kafka.common.protocol.CommonFields.ERROR_MESSAGE; @@ -62,12 +64,11 @@ public static Schema[] schemaVersions() { } private final int throttleTimeMs; - private final Map errors; + private final Map errors; - public AlterConfigsResponse(int throttleTimeMs, Map errors) { + public AlterConfigsResponse(int throttleTimeMs, Map errors) { this.throttleTimeMs = throttleTimeMs; - this.errors = errors; - + this.errors = Objects.requireNonNull(errors, "errors"); } public AlterConfigsResponse(Struct struct) { @@ -77,13 +78,13 @@ public AlterConfigsResponse(Struct struct) { for (Object resourceObj : resourcesArray) { Struct resourceStruct = (Struct) resourceObj; ApiError error = new ApiError(resourceStruct); - ResourceType resourceType = ResourceType.forId(resourceStruct.getByte(RESOURCE_TYPE_KEY_NAME)); + ConfigResource.Type resourceType = ConfigResource.Type.forId(resourceStruct.getByte(RESOURCE_TYPE_KEY_NAME)); String resourceName = resourceStruct.getString(RESOURCE_NAME_KEY_NAME); - errors.put(new Resource(resourceType, resourceName), error); + errors.put(new ConfigResource(resourceType, resourceName), error); } } - public Map errors() { + public Map errors() { return errors; } @@ -102,9 +103,9 @@ protected Struct toStruct(short version) { Struct struct = new Struct(ApiKeys.ALTER_CONFIGS.responseSchema(version)); struct.set(THROTTLE_TIME_MS, throttleTimeMs); List resourceStructs = new ArrayList<>(errors.size()); - for (Map.Entry entry : errors.entrySet()) { + for (Map.Entry entry : errors.entrySet()) { Struct resourceStruct = struct.instance(RESOURCES_KEY_NAME); - Resource resource = entry.getKey(); + ConfigResource resource = entry.getKey(); entry.getValue().write(resourceStruct); resourceStruct.set(RESOURCE_TYPE_KEY_NAME, resource.type().id()); resourceStruct.set(RESOURCE_NAME_KEY_NAME, resource.name()); diff --git a/clients/src/main/java/org/apache/kafka/common/requests/CreateAclsRequest.java b/clients/src/main/java/org/apache/kafka/common/requests/CreateAclsRequest.java index 00f65c6ca1cb2..a77a373e29659 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/CreateAclsRequest.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/CreateAclsRequest.java @@ -19,14 +19,14 @@ import org.apache.kafka.common.acl.AccessControlEntry; import org.apache.kafka.common.acl.AclBinding; +import org.apache.kafka.common.resource.ResourcePattern; import org.apache.kafka.common.errors.UnsupportedVersionException; import org.apache.kafka.common.protocol.ApiKeys; 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.Struct; -import org.apache.kafka.common.resource.Resource; -import org.apache.kafka.common.resource.ResourceNameType; +import org.apache.kafka.common.resource.PatternType; import org.apache.kafka.common.utils.Utils; import java.nio.ByteBuffer; @@ -38,7 +38,7 @@ import static org.apache.kafka.common.protocol.CommonFields.PERMISSION_TYPE; import static org.apache.kafka.common.protocol.CommonFields.PRINCIPAL; import static org.apache.kafka.common.protocol.CommonFields.RESOURCE_NAME; -import static org.apache.kafka.common.protocol.CommonFields.RESOURCE_NAME_TYPE; +import static org.apache.kafka.common.protocol.CommonFields.RESOURCE_PATTERN_TYPE; import static org.apache.kafka.common.protocol.CommonFields.RESOURCE_TYPE; public class CreateAclsRequest extends AbstractRequest { @@ -54,16 +54,16 @@ public class CreateAclsRequest extends AbstractRequest { PERMISSION_TYPE)))); /** - * Version 1 adds RESOURCE_NAME_TYPE. - * Also, when the quota is violated, brokers will respond to a version 1 or later request before throttling. + * Version 1 adds RESOURCE_PATTERN_TYPE, to support more than just literal resource patterns. + * For more info, see {@link PatternType}. * - * For more info, see {@link org.apache.kafka.common.resource.ResourceNameType}. + * Also, when the quota is violated, brokers will respond to a version 1 or later request before throttling. */ private static final Schema CREATE_ACLS_REQUEST_V1 = new Schema( new Field(CREATIONS_KEY_NAME, new ArrayOf(new Schema( RESOURCE_TYPE, RESOURCE_NAME, - RESOURCE_NAME_TYPE, + RESOURCE_PATTERN_TYPE, PRINCIPAL, HOST, OPERATION, @@ -81,9 +81,9 @@ public AclCreation(AclBinding acl) { } static AclCreation fromStruct(Struct struct) { - Resource resource = RequestUtils.resourceFromStructFields(struct); + ResourcePattern pattern = RequestUtils.resourcePatternromStructFields(struct); AccessControlEntry entry = RequestUtils.aceFromStructFields(struct); - return new AclCreation(new AclBinding(resource, entry)); + return new AclCreation(new AclBinding(pattern, entry)); } public AclBinding acl() { @@ -91,7 +91,7 @@ public AclBinding acl() { } void setStructFields(Struct struct) { - RequestUtils.resourceSetStructFields(acl.resource(), struct); + RequestUtils.resourcePatternSetStructFields(acl.pattern(), struct); RequestUtils.aceSetStructFields(acl.entry(), struct); } @@ -179,12 +179,19 @@ private void validate(List aclCreations) { if (version() == 0) { final boolean unsupported = aclCreations.stream() .map(AclCreation::acl) - .map(AclBinding::resource) - .map(Resource::nameType) - .anyMatch(nameType -> nameType != ResourceNameType.LITERAL); + .map(AclBinding::pattern) + .map(ResourcePattern::patternType) + .anyMatch(patternType -> patternType != PatternType.LITERAL); if (unsupported) { - throw new UnsupportedVersionException("Version 0 only supports literal resource name types"); + throw new UnsupportedVersionException("Version 0 only supports literal resource pattern types"); } } + + final boolean unknown = aclCreations.stream() + .map(AclCreation::acl) + .anyMatch(AclBinding::isUnknown); + if (unknown) { + throw new IllegalArgumentException("You can not create ACL bindings with unknown elements"); + } } } diff --git a/clients/src/main/java/org/apache/kafka/common/requests/DeleteAclsRequest.java b/clients/src/main/java/org/apache/kafka/common/requests/DeleteAclsRequest.java index d896bb2c88c3d..4c19a4adbedeb 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/DeleteAclsRequest.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/DeleteAclsRequest.java @@ -24,8 +24,8 @@ import org.apache.kafka.common.protocol.types.Field; import org.apache.kafka.common.protocol.types.Schema; import org.apache.kafka.common.protocol.types.Struct; -import org.apache.kafka.common.resource.ResourceFilter; -import org.apache.kafka.common.resource.ResourceNameType; +import org.apache.kafka.common.resource.PatternType; +import org.apache.kafka.common.resource.ResourcePatternFilter; import org.apache.kafka.common.utils.Utils; import java.nio.ByteBuffer; @@ -39,7 +39,7 @@ import static org.apache.kafka.common.protocol.CommonFields.PERMISSION_TYPE; import static org.apache.kafka.common.protocol.CommonFields.PRINCIPAL_FILTER; import static org.apache.kafka.common.protocol.CommonFields.RESOURCE_NAME_FILTER; -import static org.apache.kafka.common.protocol.CommonFields.RESOURCE_NAME_TYPE_FILTER; +import static org.apache.kafka.common.protocol.CommonFields.RESOURCE_PATTERN_TYPE_FILTER; import static org.apache.kafka.common.protocol.CommonFields.RESOURCE_TYPE; public class DeleteAclsRequest extends AbstractRequest { @@ -55,16 +55,16 @@ public class DeleteAclsRequest extends AbstractRequest { PERMISSION_TYPE)))); /** - * V1 sees a new `RESOURCE_NAME_TYPE_FILTER` that controls how the filter handles different resource name types. - * Also, when the quota is violated, brokers will respond to a version 1 or later request before throttling. + * V1 sees a new `RESOURCE_PATTERN_TYPE_FILTER` that controls how the filter handles different resource pattern types. + * For more info, see {@link PatternType}. * - * For more info, see {@link org.apache.kafka.common.resource.ResourceNameType}. + * Also, when the quota is violated, brokers will respond to a version 1 or later request before throttling. */ private static final Schema DELETE_ACLS_REQUEST_V1 = new Schema( new Field(FILTERS, new ArrayOf(new Schema( RESOURCE_TYPE, RESOURCE_NAME_FILTER, - RESOURCE_NAME_TYPE_FILTER, + RESOURCE_PATTERN_TYPE_FILTER, PRINCIPAL_FILTER, HOST_FILTER, OPERATION, @@ -107,7 +107,7 @@ public DeleteAclsRequest(Struct struct, short version) { this.filters = new ArrayList<>(); for (Object filterStructObj : struct.getArray(FILTERS)) { Struct filterStruct = (Struct) filterStructObj; - ResourceFilter resourceFilter = RequestUtils.resourceFilterFromStructFields(filterStruct); + ResourcePatternFilter resourceFilter = RequestUtils.resourcePatternFilterFromStructFields(filterStruct); AccessControlEntryFilter aceFilter = RequestUtils.aceFilterFromStructFields(filterStruct); this.filters.add(new AclBindingFilter(resourceFilter, aceFilter)); } @@ -123,7 +123,7 @@ protected Struct toStruct() { List filterStructs = new ArrayList<>(); for (AclBindingFilter filter : filters) { Struct filterStruct = struct.instance(FILTERS); - RequestUtils.resourceFilterSetStructFields(filter.resourceFilter(), filterStruct); + RequestUtils.resourcePatternFilterSetStructFields(filter.patternFilter(), filterStruct); RequestUtils.aceFilterSetStructFields(filter.entryFilter(), filterStruct); filterStructs.add(filterStruct); } @@ -156,12 +156,17 @@ public static DeleteAclsRequest parse(ByteBuffer buffer, short version) { private void validate(short version, List filters) { if (version == 0) { final boolean unsupported = filters.stream() - .map(AclBindingFilter::resourceFilter) - .map(ResourceFilter::nameType) - .anyMatch(nameType -> nameType != ResourceNameType.LITERAL); + .map(AclBindingFilter::patternFilter) + .map(ResourcePatternFilter::patternType) + .anyMatch(patternType -> patternType != PatternType.LITERAL && patternType != PatternType.ANY); if (unsupported) { - throw new UnsupportedVersionException("Version 0 only supports literal resource name types"); + throw new UnsupportedVersionException("Version 0 only supports literal resource pattern types"); } } + + final boolean unknown = filters.stream().anyMatch(AclBindingFilter::isUnknown); + if (unknown) { + throw new IllegalArgumentException("Filters contain UNKNOWN elements"); + } } } diff --git a/clients/src/main/java/org/apache/kafka/common/requests/DeleteAclsResponse.java b/clients/src/main/java/org/apache/kafka/common/requests/DeleteAclsResponse.java index 1790457f7e125..a3b81ccaf747b 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/DeleteAclsResponse.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/DeleteAclsResponse.java @@ -18,6 +18,7 @@ import org.apache.kafka.common.acl.AccessControlEntry; import org.apache.kafka.common.acl.AclBinding; +import org.apache.kafka.common.resource.ResourcePattern; import org.apache.kafka.common.errors.UnsupportedVersionException; import org.apache.kafka.common.protocol.ApiKeys; import org.apache.kafka.common.protocol.Errors; @@ -25,8 +26,7 @@ import org.apache.kafka.common.protocol.types.Field; import org.apache.kafka.common.protocol.types.Schema; import org.apache.kafka.common.protocol.types.Struct; -import org.apache.kafka.common.resource.Resource; -import org.apache.kafka.common.resource.ResourceNameType; +import org.apache.kafka.common.resource.PatternType; import org.apache.kafka.common.utils.Utils; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -45,7 +45,7 @@ import static org.apache.kafka.common.protocol.CommonFields.PERMISSION_TYPE; import static org.apache.kafka.common.protocol.CommonFields.PRINCIPAL; import static org.apache.kafka.common.protocol.CommonFields.RESOURCE_NAME; -import static org.apache.kafka.common.protocol.CommonFields.RESOURCE_NAME_TYPE; +import static org.apache.kafka.common.protocol.CommonFields.RESOURCE_PATTERN_TYPE; import static org.apache.kafka.common.protocol.CommonFields.RESOURCE_TYPE; import static org.apache.kafka.common.protocol.CommonFields.THROTTLE_TIME_MS; @@ -65,16 +65,16 @@ public class DeleteAclsResponse extends AbstractResponse { PERMISSION_TYPE); /** - * V1 sees a new `RESOURCE_NAME_TYPE` that describes how the resource name is interpreted. + * V1 sees a new `RESOURCE_PATTERN_TYPE` that defines the type of the resource pattern. * - * For more info, see {@link org.apache.kafka.common.resource.ResourceNameType}. + * For more info, see {@link PatternType}. */ private static final Schema MATCHING_ACL_V1 = new Schema( ERROR_CODE, ERROR_MESSAGE, RESOURCE_TYPE, RESOURCE_NAME, - RESOURCE_NAME_TYPE, + RESOURCE_PATTERN_TYPE, PRINCIPAL, HOST, OPERATION, @@ -89,10 +89,10 @@ public class DeleteAclsResponse extends AbstractResponse { new Field(MATCHING_ACLS_KEY_NAME, new ArrayOf(MATCHING_ACL_V0), "The matching ACLs"))))); /** - * V1 sees a new `RESOURCE_NAME_TYPE` field added to MATCHING_ACL_V1, that describes how the resource name is interpreted + * V1 sees a new `RESOURCE_PATTERN_TYPE` field added to MATCHING_ACL_V1, that describes how the resource pattern is interpreted * and version was bumped to indicate that, on quota violation, brokers send out responses before throttling. * - * For more info, see {@link org.apache.kafka.common.resource.ResourceNameType}. + * For more info, see {@link PatternType}. */ private static final Schema DELETE_ACLS_RESPONSE_V1 = new Schema( THROTTLE_TIME_MS, @@ -180,7 +180,7 @@ public DeleteAclsResponse(Struct struct) { Struct matchingAclStruct = (Struct) matchingAclStructObj; ApiError matchError = new ApiError(matchingAclStruct); AccessControlEntry entry = RequestUtils.aceFromStructFields(matchingAclStruct); - Resource resource = RequestUtils.resourceFromStructFields(matchingAclStruct); + ResourcePattern resource = RequestUtils.resourcePatternromStructFields(matchingAclStruct); deletions.add(new AclDeletionResult(matchError, new AclBinding(resource, entry))); } this.responses.add(new AclFilterResponse(error, deletions)); @@ -201,7 +201,7 @@ protected Struct toStruct(short version) { for (AclDeletionResult deletion : response.deletions()) { Struct deletionStruct = responseStruct.instance(MATCHING_ACLS_KEY_NAME); deletion.error.write(deletionStruct); - RequestUtils.resourceSetStructFields(deletion.acl().resource(), deletionStruct); + RequestUtils.resourcePatternSetStructFields(deletion.acl().pattern(), deletionStruct); RequestUtils.aceSetStructFields(deletion.acl().entry(), deletionStruct); deletionStructs.add(deletionStruct); } @@ -247,12 +247,20 @@ private void validate(short version) { final boolean unsupported = responses.stream() .flatMap(r -> r.deletions.stream()) .map(AclDeletionResult::acl) - .map(AclBinding::resource) - .map(Resource::nameType) - .anyMatch(nameType -> nameType != ResourceNameType.LITERAL); + .map(AclBinding::pattern) + .map(ResourcePattern::patternType) + .anyMatch(patternType -> patternType != PatternType.LITERAL); if (unsupported) { - throw new UnsupportedVersionException("Version 0 only supports literal resource name types"); + throw new UnsupportedVersionException("Version 0 only supports literal resource pattern types"); } } + + final boolean unknown = responses.stream() + .flatMap(r -> r.deletions.stream()) + .map(AclDeletionResult::acl) + .anyMatch(AclBinding::isUnknown); + if (unknown) { + throw new IllegalArgumentException("Response contains UNKNOWN elements"); + } } } diff --git a/clients/src/main/java/org/apache/kafka/common/requests/DescribeAclsRequest.java b/clients/src/main/java/org/apache/kafka/common/requests/DescribeAclsRequest.java index d3a04d09a1349..d2198397f6daa 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/DescribeAclsRequest.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/DescribeAclsRequest.java @@ -17,14 +17,13 @@ package org.apache.kafka.common.requests; import org.apache.kafka.common.acl.AccessControlEntryFilter; -import org.apache.kafka.common.acl.AclBinding; import org.apache.kafka.common.acl.AclBindingFilter; import org.apache.kafka.common.errors.UnsupportedVersionException; import org.apache.kafka.common.protocol.ApiKeys; import org.apache.kafka.common.protocol.types.Schema; import org.apache.kafka.common.protocol.types.Struct; -import org.apache.kafka.common.resource.ResourceFilter; -import org.apache.kafka.common.resource.ResourceNameType; +import org.apache.kafka.common.resource.PatternType; +import org.apache.kafka.common.resource.ResourcePatternFilter; import java.nio.ByteBuffer; import java.util.Collections; @@ -34,7 +33,7 @@ import static org.apache.kafka.common.protocol.CommonFields.PERMISSION_TYPE; import static org.apache.kafka.common.protocol.CommonFields.PRINCIPAL_FILTER; import static org.apache.kafka.common.protocol.CommonFields.RESOURCE_NAME_FILTER; -import static org.apache.kafka.common.protocol.CommonFields.RESOURCE_NAME_TYPE_FILTER; +import static org.apache.kafka.common.protocol.CommonFields.RESOURCE_PATTERN_TYPE_FILTER; import static org.apache.kafka.common.protocol.CommonFields.RESOURCE_TYPE; public class DescribeAclsRequest extends AbstractRequest { @@ -47,15 +46,15 @@ public class DescribeAclsRequest extends AbstractRequest { PERMISSION_TYPE); /** - * V1 sees a new `RESOURCE_NAME_TYPE_FILTER` that controls how the filter handles different resource name types. - * Also, when the quota is violated, brokers will respond to a version 1 or later request before throttling. + * V1 sees a new `RESOURCE_PATTERN_TYPE_FILTER` that controls how the filter handles different resource pattern types. + * For more info, see {@link PatternType}. * - * For more info, see {@link org.apache.kafka.common.resource.ResourceNameType}. + * Also, when the quota is violated, brokers will respond to a version 1 or later request before throttling. */ private static final Schema DESCRIBE_ACLS_REQUEST_V1 = new Schema( RESOURCE_TYPE, RESOURCE_NAME_FILTER, - RESOURCE_NAME_TYPE_FILTER, + RESOURCE_PATTERN_TYPE_FILTER, PRINCIPAL_FILTER, HOST_FILTER, OPERATION, @@ -95,7 +94,7 @@ public String toString() { public DescribeAclsRequest(Struct struct, short version) { super(version); - ResourceFilter resourceFilter = RequestUtils.resourceFilterFromStructFields(struct); + ResourcePatternFilter resourceFilter = RequestUtils.resourcePatternFilterFromStructFields(struct); AccessControlEntryFilter entryFilter = RequestUtils.aceFilterFromStructFields(struct); this.filter = new AclBindingFilter(resourceFilter, entryFilter); } @@ -103,7 +102,7 @@ public DescribeAclsRequest(Struct struct, short version) { @Override protected Struct toStruct() { Struct struct = new Struct(ApiKeys.DESCRIBE_ACLS.requestSchema(version())); - RequestUtils.resourceFilterSetStructFields(filter.resourceFilter(), struct); + RequestUtils.resourcePatternFilterSetStructFields(filter.patternFilter(), struct); RequestUtils.aceFilterSetStructFields(filter.entryFilter(), struct); return struct; } @@ -115,7 +114,7 @@ public AbstractResponse getErrorResponse(int throttleTimeMs, Throwable throwable case 0: case 1: return new DescribeAclsResponse(throttleTimeMs, ApiError.fromThrowable(throwable), - Collections.emptySet()); + 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(), ApiKeys.DESCRIBE_ACLS.latestVersion())); @@ -131,8 +130,14 @@ public AclBindingFilter filter() { } private void validate(AclBindingFilter filter, short version) { - if (version == 0 && filter.resourceFilter().nameType() != ResourceNameType.LITERAL) { - throw new UnsupportedVersionException("Version 0 only supports literal resource name types"); + if (version == 0 + && filter.patternFilter().patternType() != PatternType.LITERAL + && filter.patternFilter().patternType() != PatternType.ANY) { + throw new UnsupportedVersionException("Version 0 only supports literal resource pattern types"); + } + + if (filter.isUnknown()) { + throw new IllegalArgumentException("Filter contain UNKNOWN elements"); } } } diff --git a/clients/src/main/java/org/apache/kafka/common/requests/DescribeAclsResponse.java b/clients/src/main/java/org/apache/kafka/common/requests/DescribeAclsResponse.java index b6673d9b4875c..341845cf2b10a 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/DescribeAclsResponse.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/DescribeAclsResponse.java @@ -19,6 +19,8 @@ import org.apache.kafka.common.acl.AccessControlEntry; import org.apache.kafka.common.acl.AclBinding; +import org.apache.kafka.common.resource.PatternType; +import org.apache.kafka.common.resource.ResourcePattern; import org.apache.kafka.common.errors.UnsupportedVersionException; import org.apache.kafka.common.protocol.ApiKeys; import org.apache.kafka.common.protocol.Errors; @@ -26,8 +28,6 @@ import org.apache.kafka.common.protocol.types.Field; import org.apache.kafka.common.protocol.types.Schema; import org.apache.kafka.common.protocol.types.Struct; -import org.apache.kafka.common.resource.Resource; -import org.apache.kafka.common.resource.ResourceNameType; import java.nio.ByteBuffer; import java.util.ArrayList; @@ -43,7 +43,7 @@ import static org.apache.kafka.common.protocol.CommonFields.PERMISSION_TYPE; import static org.apache.kafka.common.protocol.CommonFields.PRINCIPAL; import static org.apache.kafka.common.protocol.CommonFields.RESOURCE_NAME; -import static org.apache.kafka.common.protocol.CommonFields.RESOURCE_NAME_TYPE; +import static org.apache.kafka.common.protocol.CommonFields.RESOURCE_PATTERN_TYPE; import static org.apache.kafka.common.protocol.CommonFields.RESOURCE_TYPE; import static org.apache.kafka.common.protocol.CommonFields.THROTTLE_TIME_MS; @@ -61,14 +61,14 @@ public class DescribeAclsResponse extends AbstractResponse { PERMISSION_TYPE)))); /** - * V1 sees a new `RESOURCE_NAME_TYPE` that describes how the resource name is interpreted. + * V1 sees a new `RESOURCE_PATTERN_TYPE` that defines the type of the resource pattern. * - * For more info, see {@link org.apache.kafka.common.resource.ResourceNameType}. + * For more info, see {@link PatternType}. */ private static final Schema DESCRIBE_ACLS_RESOURCE_V1 = new Schema( RESOURCE_TYPE, RESOURCE_NAME, - RESOURCE_NAME_TYPE, + RESOURCE_PATTERN_TYPE, new Field(ACLS_KEY_NAME, new ArrayOf(new Schema( PRINCIPAL, HOST, @@ -82,10 +82,10 @@ public class DescribeAclsResponse extends AbstractResponse { new Field(RESOURCES_KEY_NAME, new ArrayOf(DESCRIBE_ACLS_RESOURCE_V0), "The resources and their associated ACLs.")); /** - * V1 sees a new `RESOURCE_NAME_TYPE` field added to DESCRIBE_ACLS_RESOURCE_V1, that describes how the resource name is interpreted + * V1 sees a new `RESOURCE_PATTERN_TYPE` field added to DESCRIBE_ACLS_RESOURCE_V1, that describes how the resource name is interpreted * and version was bumped to indicate that, on quota violation, brokers send out responses before throttling. * - * For more info, see {@link org.apache.kafka.common.resource.ResourceNameType}. + * For more info, see {@link PatternType}. */ private static final Schema DESCRIBE_ACLS_RESPONSE_V1 = new Schema( THROTTLE_TIME_MS, @@ -113,11 +113,11 @@ public DescribeAclsResponse(Struct struct) { this.acls = new ArrayList<>(); for (Object resourceStructObj : struct.getArray(RESOURCES_KEY_NAME)) { Struct resourceStruct = (Struct) resourceStructObj; - Resource resource = RequestUtils.resourceFromStructFields(resourceStruct); + ResourcePattern pattern = RequestUtils.resourcePatternromStructFields(resourceStruct); for (Object aclDataStructObj : resourceStruct.getArray(ACLS_KEY_NAME)) { Struct aclDataStruct = (Struct) aclDataStructObj; AccessControlEntry entry = RequestUtils.aceFromStructFields(aclDataStruct); - this.acls.add(new AclBinding(resource, entry)); + this.acls.add(new AclBinding(pattern, entry)); } } } @@ -130,21 +130,18 @@ protected Struct toStruct(short version) { struct.set(THROTTLE_TIME_MS, throttleTimeMs); error.write(struct); - Map> resourceToData = new HashMap<>(); + Map> resourceToData = new HashMap<>(); for (AclBinding acl : acls) { - List entry = resourceToData.get(acl.resource()); - if (entry == null) { - entry = new ArrayList<>(); - resourceToData.put(acl.resource(), entry); - } - entry.add(acl.entry()); + resourceToData + .computeIfAbsent(acl.pattern(), k -> new ArrayList<>()) + .add(acl.entry()); } List resourceStructs = new ArrayList<>(); - for (Map.Entry> tuple : resourceToData.entrySet()) { - Resource resource = tuple.getKey(); + for (Map.Entry> tuple : resourceToData.entrySet()) { + ResourcePattern resource = tuple.getKey(); Struct resourceStruct = struct.instance(RESOURCES_KEY_NAME); - RequestUtils.resourceSetStructFields(resource, resourceStruct); + RequestUtils.resourcePatternSetStructFields(resource, resourceStruct); List dataStructs = new ArrayList<>(); for (AccessControlEntry entry : tuple.getValue()) { Struct dataStruct = resourceStruct.instance(ACLS_KEY_NAME); @@ -188,12 +185,17 @@ public boolean shouldClientThrottle(short version) { private void validate(short version) { if (version == 0) { final boolean unsupported = acls.stream() - .map(AclBinding::resource) - .map(Resource::nameType) - .anyMatch(nameType -> nameType != ResourceNameType.LITERAL); + .map(AclBinding::pattern) + .map(ResourcePattern::patternType) + .anyMatch(patternType -> patternType != PatternType.LITERAL); if (unsupported) { - throw new UnsupportedVersionException("Version 0 only supports literal resource name types"); + throw new UnsupportedVersionException("Version 0 only supports literal resource pattern types"); } } + + final boolean unknown = acls.stream().anyMatch(AclBinding::isUnknown); + if (unknown) { + throw new IllegalArgumentException("Contain UNKNOWN elements"); + } } } diff --git a/clients/src/main/java/org/apache/kafka/common/requests/DescribeConfigsRequest.java b/clients/src/main/java/org/apache/kafka/common/requests/DescribeConfigsRequest.java index 72bb112f497a2..781cd451b987e 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/DescribeConfigsRequest.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/DescribeConfigsRequest.java @@ -16,6 +16,7 @@ */ package org.apache.kafka.common.requests; +import org.apache.kafka.common.config.ConfigResource; import org.apache.kafka.common.protocol.ApiKeys; import org.apache.kafka.common.protocol.types.ArrayOf; import org.apache.kafka.common.protocol.types.Field; @@ -29,6 +30,7 @@ import java.util.HashMap; import java.util.List; import java.util.Map; +import java.util.Objects; import static org.apache.kafka.common.protocol.types.Type.BOOLEAN; import static org.apache.kafka.common.protocol.types.Type.INT8; @@ -64,12 +66,12 @@ public static Schema[] schemaVersions() { } public static class Builder extends AbstractRequest.Builder { - private final Map> resourceToConfigNames; + private final Map> resourceToConfigNames; private boolean includeSynonyms; - public Builder(Map> resourceToConfigNames) { + public Builder(Map> resourceToConfigNames) { super(ApiKeys.DESCRIBE_CONFIGS); - this.resourceToConfigNames = resourceToConfigNames; + this.resourceToConfigNames = Objects.requireNonNull(resourceToConfigNames, "resourceToConfigNames"); } public Builder includeSynonyms(boolean includeSynonyms) { @@ -77,13 +79,13 @@ public Builder includeSynonyms(boolean includeSynonyms) { return this; } - public Builder(Collection resources) { + public Builder(Collection resources) { this(toResourceToConfigNames(resources)); } - private static Map> toResourceToConfigNames(Collection resources) { - Map> result = new HashMap<>(resources.size()); - for (Resource resource : resources) + private static Map> toResourceToConfigNames(Collection resources) { + Map> result = new HashMap<>(resources.size()); + for (ConfigResource resource : resources) result.put(resource, null); return result; } @@ -94,12 +96,12 @@ public DescribeConfigsRequest build(short version) { } } - private final Map> resourceToConfigNames; + private final Map> resourceToConfigNames; private final boolean includeSynonyms; - public DescribeConfigsRequest(short version, Map> resourceToConfigNames, boolean includeSynonyms) { + public DescribeConfigsRequest(short version, Map> resourceToConfigNames, boolean includeSynonyms) { super(version); - this.resourceToConfigNames = resourceToConfigNames; + this.resourceToConfigNames = Objects.requireNonNull(resourceToConfigNames, "resourceToConfigNames"); this.includeSynonyms = includeSynonyms; } @@ -109,7 +111,7 @@ public DescribeConfigsRequest(Struct struct, short version) { resourceToConfigNames = new HashMap<>(resourcesArray.length); for (Object resourceObj : resourcesArray) { Struct resourceStruct = (Struct) resourceObj; - ResourceType resourceType = ResourceType.forId(resourceStruct.getByte(RESOURCE_TYPE_KEY_NAME)); + ConfigResource.Type resourceType = ConfigResource.Type.forId(resourceStruct.getByte(RESOURCE_TYPE_KEY_NAME)); String resourceName = resourceStruct.getString(RESOURCE_NAME_KEY_NAME); Object[] configNamesArray = resourceStruct.getArray(CONFIG_NAMES_KEY_NAME); @@ -120,19 +122,19 @@ public DescribeConfigsRequest(Struct struct, short version) { configNames.add((String) configNameObj); } - resourceToConfigNames.put(new Resource(resourceType, resourceName), configNames); + resourceToConfigNames.put(new ConfigResource(resourceType, resourceName), configNames); } this.includeSynonyms = struct.hasField(INCLUDE_SYNONYMS) ? struct.getBoolean(INCLUDE_SYNONYMS) : false; } - public Collection resources() { + public Collection resources() { return resourceToConfigNames.keySet(); } /** * Return null if all config names should be returned. */ - public Collection configNames(Resource resource) { + public Collection configNames(ConfigResource resource) { return resourceToConfigNames.get(resource); } @@ -144,8 +146,8 @@ public boolean includeSynonyms() { protected Struct toStruct() { Struct struct = new Struct(ApiKeys.DESCRIBE_CONFIGS.requestSchema(version())); List resourceStructs = new ArrayList<>(resources().size()); - for (Map.Entry> entry : resourceToConfigNames.entrySet()) { - Resource resource = entry.getKey(); + for (Map.Entry> entry : resourceToConfigNames.entrySet()) { + ConfigResource resource = entry.getKey(); Struct resourceStruct = struct.instance(RESOURCES_KEY_NAME); resourceStruct.set(RESOURCE_TYPE_KEY_NAME, resource.type().id()); resourceStruct.set(RESOURCE_NAME_KEY_NAME, resource.name()); @@ -168,10 +170,10 @@ public DescribeConfigsResponse getErrorResponse(int throttleTimeMs, Throwable e) case 1: case 2: ApiError error = ApiError.fromThrowable(e); - Map errors = new HashMap<>(resources().size()); + Map errors = new HashMap<>(resources().size()); DescribeConfigsResponse.Config config = new DescribeConfigsResponse.Config(error, - Collections.emptyList()); - for (Resource resource : resources()) + Collections.emptyList()); + for (ConfigResource resource : resources()) errors.put(resource, config); return new DescribeConfigsResponse(throttleTimeMs, errors); default: diff --git a/clients/src/main/java/org/apache/kafka/common/requests/DescribeConfigsResponse.java b/clients/src/main/java/org/apache/kafka/common/requests/DescribeConfigsResponse.java index 9ae1b5e1c7b22..51c35d56f22dc 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/DescribeConfigsResponse.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/DescribeConfigsResponse.java @@ -17,11 +17,12 @@ package org.apache.kafka.common.requests; +import org.apache.kafka.common.config.ConfigResource; import org.apache.kafka.common.protocol.ApiKeys; +import org.apache.kafka.common.protocol.Errors; 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.Errors; import org.apache.kafka.common.protocol.types.Struct; import java.nio.ByteBuffer; @@ -31,6 +32,7 @@ import java.util.HashMap; import java.util.List; import java.util.Map; +import java.util.Objects; import static org.apache.kafka.common.protocol.CommonFields.ERROR_CODE; import static org.apache.kafka.common.protocol.CommonFields.ERROR_MESSAGE; @@ -114,8 +116,8 @@ public static class Config { private final Collection entries; public Config(ApiError error, Collection entries) { - this.error = error; - this.entries = entries; + this.error = Objects.requireNonNull(error, "error"); + this.entries = Objects.requireNonNull(entries, "entries"); } public ApiError error() { @@ -138,12 +140,12 @@ public static class ConfigEntry { public ConfigEntry(String name, String value, ConfigSource source, boolean isSensitive, boolean readOnly, Collection synonyms) { - this.name = name; + this.name = Objects.requireNonNull(name, "name"); this.value = value; - this.source = source; + this.source = Objects.requireNonNull(source, "source"); this.isSensitive = isSensitive; this.readOnly = readOnly; - this.synonyms = synonyms; + this.synonyms = Objects.requireNonNull(synonyms, "synonyms"); } public String name() { @@ -201,9 +203,9 @@ public static class ConfigSynonym { private final ConfigSource source; public ConfigSynonym(String name, String value, ConfigSource source) { - this.name = name; + this.name = Objects.requireNonNull(name, "name"); this.value = value; - this.source = source; + this.source = Objects.requireNonNull(source, "source"); } public String name() { @@ -219,11 +221,11 @@ public ConfigSource source() { private final int throttleTimeMs; - private final Map configs; + private final Map configs; - public DescribeConfigsResponse(int throttleTimeMs, Map configs) { + public DescribeConfigsResponse(int throttleTimeMs, Map configs) { this.throttleTimeMs = throttleTimeMs; - this.configs = configs; + this.configs = Objects.requireNonNull(configs, "configs"); } public DescribeConfigsResponse(Struct struct) { @@ -234,9 +236,9 @@ public DescribeConfigsResponse(Struct struct) { Struct resourceStruct = (Struct) resourceObj; ApiError error = new ApiError(resourceStruct); - ResourceType resourceType = ResourceType.forId(resourceStruct.getByte(RESOURCE_TYPE_KEY_NAME)); + ConfigResource.Type resourceType = ConfigResource.Type.forId(resourceStruct.getByte(RESOURCE_TYPE_KEY_NAME)); String resourceName = resourceStruct.getString(RESOURCE_NAME_KEY_NAME); - Resource resource = new Resource(resourceType, resourceName); + ConfigResource resource = new ConfigResource(resourceType, resourceName); Object[] configEntriesArray = resourceStruct.getArray(CONFIG_ENTRIES_KEY_NAME); List configEntries = new ArrayList<>(configEntriesArray.length); @@ -287,11 +289,11 @@ else if (configEntriesStruct.hasField(IS_DEFAULT_KEY_NAME)) { } } - public Map configs() { + public Map configs() { return configs; } - public Config config(Resource resource) { + public Config config(ConfigResource resource) { return configs.get(resource); } @@ -313,10 +315,10 @@ protected Struct toStruct(short version) { Struct struct = new Struct(ApiKeys.DESCRIBE_CONFIGS.responseSchema(version)); struct.set(THROTTLE_TIME_MS, throttleTimeMs); List resourceStructs = new ArrayList<>(configs.size()); - for (Map.Entry entry : configs.entrySet()) { + for (Map.Entry entry : configs.entrySet()) { Struct resourceStruct = struct.instance(RESOURCES_KEY_NAME); - Resource resource = entry.getKey(); + ConfigResource resource = entry.getKey(); resourceStruct.set(RESOURCE_TYPE_KEY_NAME, resource.type().id()); resourceStruct.set(RESOURCE_NAME_KEY_NAME, resource.name()); diff --git a/clients/src/main/java/org/apache/kafka/common/requests/FindCoordinatorResponse.java b/clients/src/main/java/org/apache/kafka/common/requests/FindCoordinatorResponse.java index 39726dac6b991..bc7f654c0bb6e 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/FindCoordinatorResponse.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/FindCoordinatorResponse.java @@ -68,9 +68,11 @@ public static Schema[] schemaVersions() { /** * Possible error codes: * + * COORDINATOR_LOAD_IN_PROGRESS (14) * COORDINATOR_NOT_AVAILABLE (15) - * NOT_COORDINATOR (16) * GROUP_AUTHORIZATION_FAILED (30) + * INVALID_REQUEST (42) + * TRANSACTIONAL_ID_AUTHORIZATION_FAILED (53) */ @@ -107,6 +109,10 @@ public int throttleTimeMs() { return throttleTimeMs; } + public boolean hasError() { + return this.error != Errors.NONE; + } + public Errors error() { return error; } diff --git a/clients/src/main/java/org/apache/kafka/common/requests/ListGroupsResponse.java b/clients/src/main/java/org/apache/kafka/common/requests/ListGroupsResponse.java index b108803590f17..de1699872521a 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/ListGroupsResponse.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/ListGroupsResponse.java @@ -62,6 +62,7 @@ public static Schema[] schemaVersions() { /** * Possible error codes: * + * COORDINATOR_LOAD_IN_PROGRESS (14) * COORDINATOR_NOT_AVAILABLE (15) * AUTHORIZATION_FAILED (29) */ 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 67745cbb4cc7a..ab8c8167e60e2 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 @@ -192,8 +192,8 @@ public String toString() { // put in the purgatory (due to client throttling, it can take a while before the response is sent). // Care should be taken in methods that use this field. private volatile Map partitionRecords; - private boolean transactional = false; - private boolean idempotent = false; + private boolean hasTransactionalRecords = false; + private boolean hasIdempotentRecords = false; private ProduceRequest(short version, short acks, int timeout, Map partitionRecords, String transactionalId) { super(version); @@ -250,8 +250,8 @@ private void validateRecords(short version, MemoryRecords records) { if (iterator.hasNext()) throw new InvalidRecordException("Produce requests with version " + version + " are only allowed to " + "contain exactly one record batch"); - idempotent = entry.hasProducerId(); - transactional = entry.isTransactional(); + hasIdempotentRecords = hasIdempotentRecords || entry.hasProducerId(); + hasTransactionalRecords = hasTransactionalRecords || entry.isTransactional(); } // Note that we do not do similar validation for older versions to ensure compatibility with @@ -359,12 +359,12 @@ public String transactionalId() { return transactionalId; } - public boolean isTransactional() { - return transactional; + public boolean hasTransactionalRecords() { + return hasTransactionalRecords; } - public boolean isIdempotent() { - return idempotent; + public boolean hasIdempotentRecords() { + return hasIdempotentRecords; } /** diff --git a/clients/src/main/java/org/apache/kafka/common/requests/RequestUtils.java b/clients/src/main/java/org/apache/kafka/common/requests/RequestUtils.java index f4f00a8064134..7638c6c20a166 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/RequestUtils.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/RequestUtils.java @@ -20,11 +20,11 @@ import org.apache.kafka.common.acl.AccessControlEntryFilter; import org.apache.kafka.common.acl.AclOperation; import org.apache.kafka.common.acl.AclPermissionType; +import org.apache.kafka.common.resource.PatternType; +import org.apache.kafka.common.resource.ResourcePattern; +import org.apache.kafka.common.resource.ResourcePatternFilter; import org.apache.kafka.common.protocol.types.Struct; -import org.apache.kafka.common.resource.Resource; -import org.apache.kafka.common.resource.ResourceFilter; import org.apache.kafka.common.resource.ResourceType; -import org.apache.kafka.common.resource.ResourceNameType; import static org.apache.kafka.common.protocol.CommonFields.HOST; import static org.apache.kafka.common.protocol.CommonFields.HOST_FILTER; @@ -34,48 +34,40 @@ import static org.apache.kafka.common.protocol.CommonFields.PRINCIPAL_FILTER; import static org.apache.kafka.common.protocol.CommonFields.RESOURCE_NAME; import static org.apache.kafka.common.protocol.CommonFields.RESOURCE_NAME_FILTER; -import static org.apache.kafka.common.protocol.CommonFields.RESOURCE_NAME_TYPE; -import static org.apache.kafka.common.protocol.CommonFields.RESOURCE_NAME_TYPE_FILTER; +import static org.apache.kafka.common.protocol.CommonFields.RESOURCE_PATTERN_TYPE; +import static org.apache.kafka.common.protocol.CommonFields.RESOURCE_PATTERN_TYPE_FILTER; import static org.apache.kafka.common.protocol.CommonFields.RESOURCE_TYPE; final class RequestUtils { private RequestUtils() {} - static Resource resourceFromStructFields(Struct struct) { + static ResourcePattern resourcePatternromStructFields(Struct struct) { byte resourceType = struct.get(RESOURCE_TYPE); String name = struct.get(RESOURCE_NAME); - ResourceNameType resourceNameType = ResourceNameType.LITERAL; - if (struct.hasField(RESOURCE_NAME_TYPE)) { - resourceNameType = ResourceNameType.fromCode(struct.get(RESOURCE_NAME_TYPE)); - } - return new Resource(ResourceType.fromCode(resourceType), name, resourceNameType); + PatternType patternType = PatternType.fromCode( + struct.getOrElse(RESOURCE_PATTERN_TYPE, PatternType.LITERAL.code())); + return new ResourcePattern(ResourceType.fromCode(resourceType), name, patternType); } - static void resourceSetStructFields(Resource resource, Struct struct) { - struct.set(RESOURCE_TYPE, resource.resourceType().code()); - struct.set(RESOURCE_NAME, resource.name()); - if (struct.hasField(RESOURCE_NAME_TYPE)) { - struct.set(RESOURCE_NAME_TYPE, resource.nameType().code()); - } + static void resourcePatternSetStructFields(ResourcePattern pattern, Struct struct) { + struct.set(RESOURCE_TYPE, pattern.resourceType().code()); + struct.set(RESOURCE_NAME, pattern.name()); + struct.setIfExists(RESOURCE_PATTERN_TYPE, pattern.patternType().code()); } - static ResourceFilter resourceFilterFromStructFields(Struct struct) { + static ResourcePatternFilter resourcePatternFilterFromStructFields(Struct struct) { byte resourceType = struct.get(RESOURCE_TYPE); String name = struct.get(RESOURCE_NAME_FILTER); - ResourceNameType resourceNameType = ResourceNameType.LITERAL; - if (struct.hasField(RESOURCE_NAME_TYPE_FILTER)) { - resourceNameType = ResourceNameType.fromCode(struct.get(RESOURCE_NAME_TYPE_FILTER)); - } - return new ResourceFilter(ResourceType.fromCode(resourceType), name, resourceNameType); + PatternType patternType = PatternType.fromCode( + struct.getOrElse(RESOURCE_PATTERN_TYPE_FILTER, PatternType.LITERAL.code())); + return new ResourcePatternFilter(ResourceType.fromCode(resourceType), name, patternType); } - static void resourceFilterSetStructFields(ResourceFilter resourceFilter, Struct struct) { - struct.set(RESOURCE_TYPE, resourceFilter.resourceType().code()); - struct.set(RESOURCE_NAME_FILTER, resourceFilter.name()); - if (struct.hasField(RESOURCE_NAME_TYPE_FILTER)) { - struct.set(RESOURCE_NAME_TYPE_FILTER, resourceFilter.nameType().code()); - } + static void resourcePatternFilterSetStructFields(ResourcePatternFilter patternFilter, Struct struct) { + struct.set(RESOURCE_TYPE, patternFilter.resourceType().code()); + struct.set(RESOURCE_NAME_FILTER, patternFilter.name()); + struct.setIfExists(RESOURCE_PATTERN_TYPE_FILTER, patternFilter.patternType().code()); } static AccessControlEntry aceFromStructFields(Struct struct) { diff --git a/clients/src/main/java/org/apache/kafka/common/resource/PatternType.java b/clients/src/main/java/org/apache/kafka/common/resource/PatternType.java new file mode 100644 index 0000000000000..1233959aabe87 --- /dev/null +++ b/clients/src/main/java/org/apache/kafka/common/resource/PatternType.java @@ -0,0 +1,124 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF 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.resource; + +import org.apache.kafka.common.annotation.InterfaceStability; + +import java.util.Arrays; +import java.util.Collections; +import java.util.Map; +import java.util.function.Function; +import java.util.stream.Collectors; + +/** + * Resource pattern type. + */ +@InterfaceStability.Evolving +public enum PatternType { + /** + * Represents any PatternType which this client cannot understand, perhaps because this client is too old. + */ + UNKNOWN((byte) 0), + + /** + * In a filter, matches any resource pattern type. + */ + ANY((byte) 1), + + /** + * In a filter, will perform pattern matching. + * + * e.g. Given a filter of {@code ResourcePatternFilter(TOPIC, "payments.received", MATCH)`}, the filter match + * any {@link ResourcePattern} that matches topic 'payments.received'. This might include: + *
    + *
  • A Literal pattern with the same type and name, e.g. {@code ResourcePattern(TOPIC, "payments.received", LITERAL)}
  • + *
  • A Wildcard pattern with the same type, e.g. {@code ResourcePattern(TOPIC, "*", LITERAL)}
  • + *
  • A Prefixed pattern with the same type and where the name is a matching prefix, e.g. {@code ResourcePattern(TOPIC, "payments.", PREFIXED)}
  • + *
+ */ + MATCH((byte) 2), + + /** + * A literal resource name. + * + * A literal name defines the full name of a resource, e.g. topic with name 'foo', or group with name 'bob'. + * + * The special wildcard character {@code *} can be used to represent a resource with any name. + */ + LITERAL((byte) 3), + + /** + * A prefixed resource name. + * + * A prefixed name defines a prefix for a resource, e.g. topics with names that start with 'foo'. + */ + PREFIXED((byte) 4); + + private final static Map CODE_TO_VALUE = + Collections.unmodifiableMap( + Arrays.stream(PatternType.values()) + .collect(Collectors.toMap(PatternType::code, Function.identity())) + ); + + private final static Map NAME_TO_VALUE = + Collections.unmodifiableMap( + Arrays.stream(PatternType.values()) + .collect(Collectors.toMap(PatternType::name, Function.identity())) + ); + + private final byte code; + + PatternType(byte code) { + this.code = code; + } + + /** + * @return the code of this resource. + */ + public byte code() { + return code; + } + + /** + * @eturn whether this resource pattern type is UNKNOWN. + */ + public boolean isUnknown() { + return this == UNKNOWN; + } + + /** + * @return whether this resource pattern type is a concrete type, rather than UNKNOWN or one of the filter types. + */ + public boolean isSpecific() { + return this != UNKNOWN && this != ANY && this != MATCH; + } + + /** + * Return the PatternType with the provided code or {@link #UNKNOWN} if one cannot be found. + */ + public static PatternType fromCode(byte code) { + return CODE_TO_VALUE.getOrDefault(code, UNKNOWN); + } + + /** + * Return the PatternType with the provided name or {@link #UNKNOWN} if one cannot be found. + */ + public static PatternType fromString(String name) { + return NAME_TO_VALUE.getOrDefault(name, UNKNOWN); + } +} diff --git a/clients/src/main/java/org/apache/kafka/common/resource/Resource.java b/clients/src/main/java/org/apache/kafka/common/resource/Resource.java index a4810b27ac4ad..f41f41a04b626 100644 --- a/clients/src/main/java/org/apache/kafka/common/resource/Resource.java +++ b/clients/src/main/java/org/apache/kafka/common/resource/Resource.java @@ -22,20 +22,14 @@ import java.util.Objects; /** - * Represents a cluster resource with a tuple of (type, name, nameType). + * Represents a cluster resource with a tuple of (type, name). * * The API for this class is still evolving and we may break compatibility in minor releases, if necessary. */ @InterfaceStability.Evolving public class Resource { - /** - * A special literal resource name that corresponds to 'all resources of a certain type'. - */ - public static final String WILDCARD_RESOURCE = "*"; - private final ResourceType resourceType; private final String name; - private final ResourceNameType nameType; /** * The name of the CLUSTER resource. @@ -45,32 +39,19 @@ public class Resource { /** * A resource representing the whole cluster. */ - public final static Resource CLUSTER = new Resource(ResourceType.CLUSTER, CLUSTER_NAME, ResourceNameType.LITERAL); - - /** - * Create an instance of this class with the provided parameters. - * - * @param resourceType non-null resource type - * @param name non-null resource name - * @param nameType non-null resource name type - */ - public Resource(ResourceType resourceType, String name, ResourceNameType nameType) { - this.resourceType = Objects.requireNonNull(resourceType, "resourceType"); - this.name = Objects.requireNonNull(name, "name"); - this.nameType = Objects.requireNonNull(nameType, "nameType"); - } + public final static Resource CLUSTER = new Resource(ResourceType.CLUSTER, CLUSTER_NAME); /** * Create an instance of this class with the provided parameters. - * Resource name type would default to ResourceNameType.LITERAL. * * @param resourceType non-null resource type * @param name non-null resource name - * @deprecated Since 2.0. Use {@link #Resource(ResourceType, String, ResourceNameType)} */ - @Deprecated public Resource(ResourceType resourceType, String name) { - this(resourceType, name, ResourceNameType.LITERAL); + Objects.requireNonNull(resourceType); + this.resourceType = resourceType; + Objects.requireNonNull(name); + this.name = name; } /** @@ -80,13 +61,6 @@ public ResourceType resourceType() { return resourceType; } - /** - * Return the resource name type. - */ - public ResourceNameType nameType() { - return nameType; - } - /** * Return the resource name. */ @@ -98,36 +72,31 @@ public String name() { * Create a filter which matches only this Resource. */ public ResourceFilter toFilter() { - return new ResourceFilter(resourceType, name, nameType); + return new ResourceFilter(resourceType, name); } @Override public String toString() { - return "(resourceType=" + resourceType + ", name=" + ((name == null) ? "" : name) + ", nameType=" + nameType + ")"; + return "(resourceType=" + resourceType + ", name=" + ((name == null) ? "" : name) + ")"; } /** * Return true if this Resource has any UNKNOWN components. */ public boolean isUnknown() { - return resourceType.isUnknown() || nameType.isUnknown(); + return resourceType.isUnknown(); } @Override public boolean equals(Object o) { - if (this == o) - return true; - if (o == null || getClass() != o.getClass()) + if (!(o instanceof Resource)) return false; - - final Resource resource = (Resource) o; - return resourceType == resource.resourceType && - Objects.equals(name, resource.name) && - nameType == resource.nameType; + Resource other = (Resource) o; + return resourceType.equals(other.resourceType) && Objects.equals(name, other.name); } @Override public int hashCode() { - return Objects.hash(resourceType, name, nameType); + return Objects.hash(resourceType, name); } } diff --git a/clients/src/main/java/org/apache/kafka/common/resource/ResourceFilter.java b/clients/src/main/java/org/apache/kafka/common/resource/ResourceFilter.java index e197e918c26c9..0a4611f9874b2 100644 --- a/clients/src/main/java/org/apache/kafka/common/resource/ResourceFilter.java +++ b/clients/src/main/java/org/apache/kafka/common/resource/ResourceFilter.java @@ -21,8 +21,6 @@ import java.util.Objects; -import static org.apache.kafka.common.resource.Resource.WILDCARD_RESOURCE; - /** * A filter which matches Resource objects. * @@ -32,54 +30,22 @@ public class ResourceFilter { private final ResourceType resourceType; private final String name; - private final ResourceNameType nameType; /** * Matches any resource. */ - public static final ResourceFilter ANY = new ResourceFilter(ResourceType.ANY, null, ResourceNameType.ANY); + public static final ResourceFilter ANY = new ResourceFilter(ResourceType.ANY, null); /** - * Create a filter that matches {@link ResourceNameType#LITERAL literal} resources of the - * supplied {@code resourceType} and {@code name}. + * Create an instance of this class with the provided parameters. * * @param resourceType non-null resource type - * @param name resource name or {@code null}. - * If {@code null}, the filter will ignore the name of resources. - * @deprecated Since 2.0. Use {@link #ResourceFilter(ResourceType, String, ResourceNameType)} + * @param name resource name or null */ - @Deprecated public ResourceFilter(ResourceType resourceType, String name) { - this(resourceType, name, ResourceNameType.LITERAL); - } - - /** - * Create a filter that matches resources of the supplied {@code resourceType}, {@code name} and - * {@code nameType}. - *

- * If the filter has each three parameters fully supplied, then it will only match a resource that has exactly - * the same values, e.g. a filter of {@code new ResourceFilter(ResourceType.GROUP, "one", ResourceTypeName.PREFIXED)} - * will only match the resource {@code new Resource(ResourceType.GROUP, "one", ResourceTypeName.PREFIXED)}. - *

- * Any of the three parameters can be set to be ignored by the filter: - *

    - *
  • {@code resourceType} can be set to {@link ResourceType#ANY}, - * meaning it will match a resource of any resource type
  • - *
  • {@code name} can be set to {@code null}, meaning it will match a resource of any name.
  • - *
  • {@code nameType} can be set to {@link ResourceNameType#ANY}, - * meaning it will match a resource with any resource name type, including the - * {@link Resource#WILDCARD_RESOURCE wildcard resource}
  • - *
- * - * @param resourceType non-null resource type to filter by. - * @param name resource name to filter by, or {@code null}. - * If {@code null}, the filter will ignore the name of resources. - * @param nameType non-null resource name type to filter by. - */ - public ResourceFilter(ResourceType resourceType, String name, ResourceNameType nameType) { - this.resourceType = Objects.requireNonNull(resourceType, "resourceType"); + Objects.requireNonNull(resourceType); + this.resourceType = resourceType; this.name = name; - this.nameType = Objects.requireNonNull(nameType, "nameType"); } /** @@ -96,76 +62,40 @@ public String name() { return name; } - /** - * Return the resource name type. - */ - public ResourceNameType nameType() { - return nameType; - } - @Override public String toString() { - return "(resourceType=" + resourceType + ", name=" + ((name == null) ? "" : name) + ", nameType=" + nameType + ")"; + return "(resourceType=" + resourceType + ", name=" + ((name == null) ? "" : name) + ")"; } /** * Return true if this ResourceFilter has any UNKNOWN components. */ public boolean isUnknown() { - return resourceType.isUnknown() || nameType.isUnknown(); + return resourceType.isUnknown(); } @Override public boolean equals(Object o) { - if (this == o) - return true; - if (o == null || getClass() != o.getClass()) + if (!(o instanceof ResourceFilter)) return false; - - final ResourceFilter that = (ResourceFilter) o; - return resourceType == that.resourceType && - Objects.equals(name, that.name) && - nameType == that.nameType; + ResourceFilter other = (ResourceFilter) o; + return resourceType.equals(other.resourceType) && Objects.equals(name, other.name); } @Override public int hashCode() { - return Objects.hash(resourceType, name, nameType); + return Objects.hash(resourceType, name); } /** * Return true if this filter matches the given Resource. - * @param other the resource path under which ACLs are stored. */ - public boolean matches(final Resource other) { - throwOnInvalidParams(other); - - if (!resourceType().equals(ResourceType.ANY) && !resourceType().equals(other.resourceType())) { + public boolean matches(Resource other) { + if ((name != null) && (!name.equals(other.name()))) return false; - } - - if (!nameType().equals(ResourceNameType.ANY) && !nameType().equals(other.nameType())) { + if ((resourceType != ResourceType.ANY) && (!resourceType.equals(other.resourceType()))) return false; - } - - if (name() == null) { - return true; - } - - if (nameType().equals(other.nameType())) { - return other.name().equals(name()); - } - - switch (other.nameType()) { - case LITERAL: - return other.name().equals(name()) || other.name().equals(WILDCARD_RESOURCE); - - case PREFIXED: - return name().startsWith(other.name()); - - default: - throw new IllegalArgumentException("Unsupported ResourceNameType: " + other.nameType()); - } + return true; } /** @@ -185,20 +115,6 @@ public String findIndefiniteField() { return "Resource type is UNKNOWN."; if (name == null) return "Resource name is NULL."; - if (nameType == ResourceNameType.ANY) - return "Resource name type is ANY."; - if (nameType == ResourceNameType.UNKNOWN) - return "Resource name type is UNKNOWN."; return null; } - - private static void throwOnInvalidParams(final Resource aclPath) { - if (aclPath.resourceType().equals(ResourceType.ANY) || aclPath.resourceType().equals(ResourceType.UNKNOWN)) { - throw new IllegalArgumentException("Only concrete resource types are supported. Got: " + aclPath.resourceType()); - } - - if (aclPath.nameType().equals(ResourceNameType.ANY) || aclPath.nameType().equals(ResourceNameType.UNKNOWN)) { - throw new IllegalArgumentException("Only concrete resource name types are supported. Got: " + aclPath.nameType()); - } - } } diff --git a/clients/src/main/java/org/apache/kafka/common/resource/ResourceNameType.java b/clients/src/main/java/org/apache/kafka/common/resource/ResourceNameType.java deleted file mode 100644 index 7aa72170de973..0000000000000 --- a/clients/src/main/java/org/apache/kafka/common/resource/ResourceNameType.java +++ /dev/null @@ -1,91 +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.common.resource; - -import org.apache.kafka.common.annotation.InterfaceStability; - -import java.util.Arrays; -import java.util.Collections; -import java.util.Map; -import java.util.function.Function; -import java.util.stream.Collectors; - -/** - * Resource name type. - */ -@InterfaceStability.Evolving -public enum ResourceNameType { - /** - * Represents any ResourceNameType which this client cannot understand, perhaps because this client is too old. - */ - UNKNOWN((byte) 0), - - /** - * In a filter, matches any resource name type. - */ - ANY((byte) 1), - - /** - * A literal resource name. - * - * A literal name defines the full name of a resource, e.g. topic with name 'foo', or group with name 'bob'. - * - * The special wildcard character {@code *} can be used to represent a resource with any name. - */ - LITERAL((byte) 2), - - /** - * A prefixed resource name. - * - * A prefixed name defines a prefix for a resource, e.g. topics with names that start with 'foo'. - */ - PREFIXED((byte) 3); - - private final static Map CODE_TO_VALUE = - Collections.unmodifiableMap( - Arrays.stream(ResourceNameType.values()) - .collect(Collectors.toMap(ResourceNameType::code, Function.identity())) - ); - - private final byte code; - - ResourceNameType(byte code) { - this.code = code; - } - - /** - * @return the code of this resource. - */ - public byte code() { - return code; - } - - /** - * Return whether this resource name type is UNKNOWN. - */ - public boolean isUnknown() { - return this == UNKNOWN; - } - - /** - * Return the ResourceNameType with the provided code or {@link #UNKNOWN} if one cannot be found. - */ - public static ResourceNameType fromCode(byte code) { - return CODE_TO_VALUE.getOrDefault(code, UNKNOWN); - } -} diff --git a/clients/src/main/java/org/apache/kafka/common/resource/ResourcePattern.java b/clients/src/main/java/org/apache/kafka/common/resource/ResourcePattern.java new file mode 100644 index 0000000000000..2b7504f70a598 --- /dev/null +++ b/clients/src/main/java/org/apache/kafka/common/resource/ResourcePattern.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.resource; + +import org.apache.kafka.common.annotation.InterfaceStability; + +import java.util.Objects; + +/** + * Represents a pattern that is used by ACLs to match zero or more + * {@link org.apache.kafka.common.resource.Resource Resources}. + * + * The API for this class is still evolving and we may break compatibility in minor releases, if necessary. + */ +@InterfaceStability.Evolving +public class ResourcePattern { + /** + * A special literal resource name that corresponds to 'all resources of a certain type'. + */ + public static final String WILDCARD_RESOURCE = "*"; + + private final ResourceType resourceType; + private final String name; + private final PatternType patternType; + + /** + * Create a pattern using the supplied parameters. + * + * @param resourceType non-null, specific, resource type + * @param name non-null resource name, which can be the {@link #WILDCARD_RESOURCE}. + * @param patternType non-null, specific, resource pattern type, which controls how the pattern will match resource names. + */ + public ResourcePattern(ResourceType resourceType, String name, PatternType patternType) { + this.resourceType = Objects.requireNonNull(resourceType, "resourceType"); + this.name = Objects.requireNonNull(name, "name"); + this.patternType = Objects.requireNonNull(patternType, "patternType"); + + if (resourceType == ResourceType.ANY) { + throw new IllegalArgumentException("resourceType must not be ANY"); + } + + if (patternType == PatternType.MATCH || patternType == PatternType.ANY) { + throw new IllegalArgumentException("patternType must not be " + patternType); + } + } + + /** + * @return the specific resource type this pattern matches + */ + public ResourceType resourceType() { + return resourceType; + } + + /** + * @return the resource name. + */ + public String name() { + return name; + } + + /** + * @return the resource pattern type. + */ + public PatternType patternType() { + return patternType; + } + + /** + * @return a filter which matches only this pattern. + */ + public ResourcePatternFilter toFilter() { + return new ResourcePatternFilter(resourceType, name, patternType); + } + + @Override + public String toString() { + return "ResourcePattern(resourceType=" + resourceType + ", name=" + ((name == null) ? "" : name) + ", patternType=" + patternType + ")"; + } + + /** + * @return {@code true} if this Resource has any UNKNOWN components. + */ + public boolean isUnknown() { + return resourceType.isUnknown() || patternType.isUnknown(); + } + + @Override + public boolean equals(Object o) { + if (this == o) + return true; + if (o == null || getClass() != o.getClass()) + return false; + + final ResourcePattern resource = (ResourcePattern) o; + return resourceType == resource.resourceType && + Objects.equals(name, resource.name) && + patternType == resource.patternType; + } + + @Override + public int hashCode() { + return Objects.hash(resourceType, name, patternType); + } +} diff --git a/clients/src/main/java/org/apache/kafka/common/resource/ResourcePatternFilter.java b/clients/src/main/java/org/apache/kafka/common/resource/ResourcePatternFilter.java new file mode 100644 index 0000000000000..83f5c88c90ddb --- /dev/null +++ b/clients/src/main/java/org/apache/kafka/common/resource/ResourcePatternFilter.java @@ -0,0 +1,170 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF 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.resource; + +import org.apache.kafka.common.annotation.InterfaceStability; + +import java.util.Objects; + +import static org.apache.kafka.common.resource.ResourcePattern.WILDCARD_RESOURCE; + +/** + * Represents a filter that can match {@link ResourcePattern}. + *

+ * The API for this class is still evolving and we may break compatibility in minor releases, if necessary. + */ +@InterfaceStability.Evolving +public class ResourcePatternFilter { + /** + * Matches any resource pattern. + */ + public static final ResourcePatternFilter ANY = new ResourcePatternFilter(ResourceType.ANY, null, PatternType.ANY); + + private final ResourceType resourceType; + private final String name; + private final PatternType patternType; + + /** + * Create a filter using the supplied parameters. + * + * @param resourceType non-null resource type. + * If {@link ResourceType#ANY}, the filter will ignore the resource type of the pattern. + * If any other resource type, the filter will match only patterns with the same type. + * @param name resource name or {@code null}. + * If {@code null}, the filter will ignore the name of resources. + * If {@link ResourcePattern#WILDCARD_RESOURCE}, will match only wildcard patterns. + * @param patternType non-null resource pattern type. + * If {@link PatternType#ANY}, the filter will match patterns regardless of pattern type. + * If {@link PatternType#MATCH}, the filter will match patterns that would match the supplied + * {@code name}, including a matching prefixed and wildcards patterns. + * If any other resource pattern type, the filter will match only patterns with the same type. + */ + public ResourcePatternFilter(ResourceType resourceType, String name, PatternType patternType) { + this.resourceType = Objects.requireNonNull(resourceType, "resourceType"); + this.name = name; + this.patternType = Objects.requireNonNull(patternType, "patternType"); + } + + /** + * @return {@code true} if this filter has any UNKNOWN components. + */ + public boolean isUnknown() { + return resourceType.isUnknown() || patternType.isUnknown(); + } + + /** + * @return the specific resource type this pattern matches + */ + public ResourceType resourceType() { + return resourceType; + } + + /** + * @return the resource name. + */ + public String name() { + return name; + } + + /** + * @return the resource pattern type. + */ + public PatternType patternType() { + return patternType; + } + + /** + * @return {@code true} if this filter matches the given pattern. + */ + public boolean matches(ResourcePattern pattern) { + if (!resourceType.equals(ResourceType.ANY) && !resourceType.equals(pattern.resourceType())) { + return false; + } + + if (!patternType.equals(PatternType.ANY) && !patternType.equals(PatternType.MATCH) && !patternType.equals(pattern.patternType())) { + return false; + } + + if (name == null) { + return true; + } + + if (patternType.equals(PatternType.ANY) || patternType.equals(pattern.patternType())) { + return name.equals(pattern.name()); + } + + switch (pattern.patternType()) { + case LITERAL: + return name.equals(pattern.name()) || pattern.name().equals(WILDCARD_RESOURCE); + + case PREFIXED: + return name.startsWith(pattern.name()); + + default: + throw new IllegalArgumentException("Unsupported PatternType: " + pattern.patternType()); + } + } + + /** + * @return {@code true} if this filter could only match one pattern. + * In other words, if there are no ANY or UNKNOWN fields. + */ + public boolean matchesAtMostOne() { + return findIndefiniteField() == null; + } + + /** + * @return a string describing any ANY or UNKNOWN field, or null if there is no such field. + */ + public String findIndefiniteField() { + if (resourceType == ResourceType.ANY) + return "Resource type is ANY."; + if (resourceType == ResourceType.UNKNOWN) + return "Resource type is UNKNOWN."; + if (name == null) + return "Resource name is NULL."; + if (patternType == PatternType.MATCH) + return "Resource pattern type is ANY."; + if (patternType == PatternType.UNKNOWN) + return "Resource pattern type is UNKNOWN."; + return null; + } + + @Override + public String toString() { + return "ResourcePattern(resourceType=" + resourceType + ", name=" + ((name == null) ? "" : name) + ", patternType=" + patternType + ")"; + } + + @Override + public boolean equals(Object o) { + if (this == o) + return true; + if (o == null || getClass() != o.getClass()) + return false; + + final ResourcePatternFilter resource = (ResourcePatternFilter) o; + return resourceType == resource.resourceType && + Objects.equals(name, resource.name) && + patternType == resource.patternType; + } + + @Override + public int hashCode() { + return Objects.hash(resourceType, name, patternType); + } +} diff --git a/clients/src/main/java/org/apache/kafka/common/security/authenticator/SaslClientAuthenticator.java b/clients/src/main/java/org/apache/kafka/common/security/authenticator/SaslClientAuthenticator.java index 2ef6d77f13f36..8934e8e548702 100644 --- a/clients/src/main/java/org/apache/kafka/common/security/authenticator/SaslClientAuthenticator.java +++ b/clients/src/main/java/org/apache/kafka/common/security/authenticator/SaslClientAuthenticator.java @@ -42,6 +42,7 @@ import org.apache.kafka.common.requests.SaslHandshakeResponse; import org.apache.kafka.common.security.auth.AuthenticateCallbackHandler; import org.apache.kafka.common.security.auth.KafkaPrincipal; +import org.apache.kafka.common.security.kerberos.KerberosError; import org.apache.kafka.common.utils.Utils; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -360,11 +361,9 @@ public byte[] run() throws SaslException { }); } catch (PrivilegedActionException e) { String error = "An error: (" + e + ") occurred when evaluating SASL token received from the Kafka Broker."; + KerberosError kerberosError = KerberosError.fromException(e); // Try to provide hints to use about what went wrong so they can fix their configuration. - // TODO: introspect about e: look for GSS information. - final String unknownServerErrorText = - "(Mechanism level: Server not found in Kerberos database (7) - UNKNOWN_SERVER)"; - if (e.toString().contains(unknownServerErrorText)) { + if (kerberosError == KerberosError.SERVER_NOT_FOUND) { error += " This may be caused by Java's being unable to resolve the Kafka Broker's" + " hostname correctly. You may want to try to adding" + " '-Dsun.net.spi.nameservice.provider.1=dns,sun' to your client's JVMFLAGS environment." + @@ -373,7 +372,13 @@ public byte[] run() throws SaslException { } error += " Kafka Client will go to AUTHENTICATION_FAILED state."; //Unwrap the SaslException inside `PrivilegedActionException` - throw new SaslAuthenticationException(error, e.getCause()); + Throwable cause = e.getCause(); + // Treat transient Kerberos errors as non-fatal SaslExceptions that are processed as I/O exceptions + // and all other failures as fatal SaslAuthenticationException. + if (kerberosError != null && kerberosError.retriable()) + throw new SaslException(error, cause); + else + throw new SaslAuthenticationException(error, cause); } } @@ -436,4 +441,5 @@ static final String firstPrincipal(Subject subject) { throw new KafkaException("Principal could not be determined from Subject, this may be a transient failure due to Kerberos re-login"); } } + } diff --git a/clients/src/main/java/org/apache/kafka/common/security/authenticator/SaslServerAuthenticator.java b/clients/src/main/java/org/apache/kafka/common/security/authenticator/SaslServerAuthenticator.java index a3f81629bc797..e8f77a53e22c7 100644 --- a/clients/src/main/java/org/apache/kafka/common/security/authenticator/SaslServerAuthenticator.java +++ b/clients/src/main/java/org/apache/kafka/common/security/authenticator/SaslServerAuthenticator.java @@ -49,6 +49,7 @@ import org.apache.kafka.common.security.auth.KafkaPrincipal; import org.apache.kafka.common.security.auth.KafkaPrincipalBuilder; import org.apache.kafka.common.security.auth.SaslAuthenticationContext; +import org.apache.kafka.common.security.kerberos.KerberosError; import org.apache.kafka.common.security.kerberos.KerberosName; import org.apache.kafka.common.security.kerberos.KerberosShortNamer; import org.apache.kafka.common.security.scram.ScramLoginModule; @@ -267,11 +268,9 @@ public void authenticate() throws IOException { default: break; } - } catch (SaslException | AuthenticationException e) { + } catch (AuthenticationException e) { // Exception will be propagated after response is sent to client - AuthenticationException authException = (e instanceof AuthenticationException) ? - (AuthenticationException) e : new AuthenticationException("SASL authentication failed", e); - setSaslState(SaslState.FAILED, authException); + setSaslState(SaslState.FAILED, e); } catch (Exception e) { // In the case of IOExceptions and other unexpected exceptions, fail immediately saslState = SaslState.FAILED; @@ -378,12 +377,20 @@ private void handleSaslToken(byte[] clientToken) throws IOException { // For versions with SASL_AUTHENTICATE header, send a response to SASL_AUTHENTICATE request even if token is empty. ByteBuffer responseBuf = responseToken == null ? EMPTY_BUFFER : ByteBuffer.wrap(responseToken); sendKafkaResponse(requestContext, new SaslAuthenticateResponse(Errors.NONE, null, responseBuf)); - } catch (SaslAuthenticationException | SaslException e) { - String errorMessage = e instanceof SaslAuthenticationException ? e.getMessage() : - "Authentication failed due to invalid credentials with SASL mechanism " + saslMechanism; - sendKafkaResponse(requestContext, new SaslAuthenticateResponse(Errors.SASL_AUTHENTICATION_FAILED, - errorMessage)); + } catch (SaslAuthenticationException e) { + sendKafkaResponse(requestContext, new SaslAuthenticateResponse(Errors.SASL_AUTHENTICATION_FAILED, e.getMessage())); throw e; + } catch (SaslException e) { + KerberosError kerberosError = KerberosError.fromException(e); + if (kerberosError != null && kerberosError.retriable()) { + // Handle retriable Kerberos exceptions as I/O exceptions rather than authentication exceptions + throw e; + } else { + String errorMessage = "Authentication failed due to invalid credentials with SASL mechanism " + saslMechanism; + sendKafkaResponse(requestContext, new SaslAuthenticateResponse(Errors.SASL_AUTHENTICATION_FAILED, + errorMessage)); + throw new SaslAuthenticationException(errorMessage, e); + } } } } diff --git a/clients/src/main/java/org/apache/kafka/common/security/kerberos/KerberosError.java b/clients/src/main/java/org/apache/kafka/common/security/kerberos/KerberosError.java new file mode 100644 index 0000000000000..c6be441c7ed82 --- /dev/null +++ b/clients/src/main/java/org/apache/kafka/common/security/kerberos/KerberosError.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.security.kerberos; + +import org.apache.kafka.common.KafkaException; +import org.apache.kafka.common.security.authenticator.SaslClientAuthenticator; +import org.apache.kafka.common.utils.Java; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import javax.security.sasl.SaslClient; +import java.lang.reflect.Method; + +/** + * Kerberos exceptions that may require special handling. The standard Kerberos error codes + * for these errors are retrieved using KrbException#errorCode() from the underlying Kerberos + * exception thrown during {@link SaslClient#evaluateChallenge(byte[])}. + */ +public enum KerberosError { + // (Mechanism level: Server not found in Kerberos database (7) - UNKNOWN_SERVER) + // This is retriable, but included here to add extra logging for this case. + SERVER_NOT_FOUND(7, false), + // (Mechanism level: Client not yet valid - try again later (21)) + CLIENT_NOT_YET_VALID(21, true), + // (Mechanism level: Ticket not yet valid (33) - Ticket not yet valid)]) + // This could be a small timing window. + TICKET_NOT_YET_VALID(33, true), + // (Mechanism level: Request is a replay (34) - Request is a replay) + // Replay detection used to prevent DoS attacks can result in false positives, so retry on error. + REPLAY(34, true); + + private static final Logger log = LoggerFactory.getLogger(SaslClientAuthenticator.class); + private static final Class KRB_EXCEPTION_CLASS; + private static final Method KRB_EXCEPTION_RETURN_CODE_METHOD; + + static { + try { + if (Java.isIbmJdk()) { + KRB_EXCEPTION_CLASS = Class.forName("com.ibm.security.krb5.internal.KrbException"); + } else { + KRB_EXCEPTION_CLASS = Class.forName("sun.security.krb5.KrbException"); + } + KRB_EXCEPTION_RETURN_CODE_METHOD = KRB_EXCEPTION_CLASS.getMethod("returnCode"); + } catch (Exception e) { + throw new KafkaException("Kerberos exceptions could not be initialized", e); + } + } + + private final int errorCode; + private final boolean retriable; + + KerberosError(int errorCode, boolean retriable) { + this.errorCode = errorCode; + this.retriable = retriable; + } + + public boolean retriable() { + return retriable; + } + + public static KerberosError fromException(Exception exception) { + Throwable cause = exception.getCause(); + while (cause != null && !KRB_EXCEPTION_CLASS.isInstance(cause)) { + cause = cause.getCause(); + } + if (cause == null) + return null; + else { + try { + Integer errorCode = (Integer) KRB_EXCEPTION_RETURN_CODE_METHOD.invoke(cause); + return fromErrorCode(errorCode); + } catch (Exception e) { + log.trace("Kerberos return code could not be determined from {} due to {}", exception, e); + return null; + } + } + } + + private static KerberosError fromErrorCode(int errorCode) { + for (KerberosError error : values()) { + if (error.errorCode == errorCode) + return error; + } + return null; + } +} diff --git a/clients/src/main/java/org/apache/kafka/common/security/oauthbearer/internals/OAuthBearerClientInitialResponse.java b/clients/src/main/java/org/apache/kafka/common/security/oauthbearer/internals/OAuthBearerClientInitialResponse.java new file mode 100644 index 0000000000000..8d4b18aede601 --- /dev/null +++ b/clients/src/main/java/org/apache/kafka/common/security/oauthbearer/internals/OAuthBearerClientInitialResponse.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.common.security.oauthbearer.internals; + +import org.apache.kafka.common.utils.Utils; + +import javax.security.sasl.SaslException; +import java.nio.charset.StandardCharsets; +import java.util.HashMap; +import java.util.Map; +import java.util.regex.Matcher; +import java.util.regex.Pattern; + +public class OAuthBearerClientInitialResponse { + static final String SEPARATOR = "\u0001"; + + private static final String SASLNAME = "(?:[\\x01-\\x7F&&[^=,]]|=2C|=3D)+"; + private static final String KEY = "[A-Za-z]+"; + private static final String VALUE = "[\\x21-\\x7E \t\r\n]+"; + private static final String KVPAIRS = String.format("(%s=%s%s)*", KEY, VALUE, SEPARATOR); + private static final Pattern AUTH_PATTERN = Pattern.compile("(?[\\w]+)[ ]+(?[-_\\.a-zA-Z0-9]+)"); + private static final Pattern CLIENT_INITIAL_RESPONSE_PATTERN = Pattern.compile( + String.format("n,(a=(?%s))?,%s(?%s)%s", SASLNAME, SEPARATOR, KVPAIRS, SEPARATOR)); + private static final String AUTH_KEY = "auth"; + + private final String tokenValue; + private final String authorizationId; + private final Map properties; + + public OAuthBearerClientInitialResponse(byte[] response) throws SaslException { + String responseMsg = new String(response, StandardCharsets.UTF_8); + Matcher matcher = CLIENT_INITIAL_RESPONSE_PATTERN.matcher(responseMsg); + if (!matcher.matches()) + throw new SaslException("Invalid OAUTHBEARER client first message"); + String authzid = matcher.group("authzid"); + this.authorizationId = authzid == null ? "" : authzid; + String kvPairs = matcher.group("kvpairs"); + this.properties = Utils.parseMap(kvPairs, "=", SEPARATOR); + String auth = properties.get(AUTH_KEY); + if (auth == null) + throw new SaslException("Invalid OAUTHBEARER client first message: 'auth' not specified"); + + Matcher authMatcher = AUTH_PATTERN.matcher(auth); + if (!authMatcher.matches()) + throw new SaslException("Invalid OAUTHBEARER client first message: invalid 'auth' format"); + if (!"bearer".equalsIgnoreCase(authMatcher.group("scheme"))) { + String msg = String.format("Invalid scheme in OAUTHBEARER client first message: %s", + matcher.group("scheme")); + throw new SaslException(msg); + } + this.tokenValue = authMatcher.group("token"); + } + + public OAuthBearerClientInitialResponse(String tokenValue) { + this(tokenValue, "", new HashMap<>()); + } + + public OAuthBearerClientInitialResponse(String tokenValue, String authorizationId, Map props) { + this.tokenValue = tokenValue; + this.authorizationId = authorizationId == null ? "" : authorizationId; + this.properties = new HashMap<>(props); + } + + public byte[] toBytes() { + String authzid = authorizationId.isEmpty() ? "" : "a=" + authorizationId; + String message = String.format("n,%s,%sauth=Bearer %s%s%s", authzid, + SEPARATOR, tokenValue, SEPARATOR, SEPARATOR); + return message.getBytes(StandardCharsets.UTF_8); + } + + public String tokenValue() { + return tokenValue; + } + + public String authorizationId() { + return authorizationId; + } + + public String propertyValue(String name) { + return properties.get(name); + } +} diff --git a/clients/src/main/java/org/apache/kafka/common/security/oauthbearer/internals/OAuthBearerSaslClient.java b/clients/src/main/java/org/apache/kafka/common/security/oauthbearer/internals/OAuthBearerSaslClient.java index 66942ba96bd17..4d4ee57b3a8d5 100644 --- a/clients/src/main/java/org/apache/kafka/common/security/oauthbearer/internals/OAuthBearerSaslClient.java +++ b/clients/src/main/java/org/apache/kafka/common/security/oauthbearer/internals/OAuthBearerSaslClient.java @@ -88,8 +88,7 @@ public byte[] evaluateChallenge(byte[] challenge) throws SaslException { throw new SaslException("Expected empty challenge"); callbackHandler().handle(new Callback[] {callback}); setState(State.RECEIVE_SERVER_FIRST_MESSAGE); - return String.format("n,,auth=Bearer %s", callback.token().value()) - .getBytes(StandardCharsets.UTF_8); + return new OAuthBearerClientInitialResponse(callback.token().value()).toBytes(); case RECEIVE_SERVER_FIRST_MESSAGE: if (challenge != null && challenge.length != 0) { String jsonErrorResponse = new String(challenge, StandardCharsets.UTF_8); diff --git a/clients/src/main/java/org/apache/kafka/common/security/oauthbearer/internals/OAuthBearerSaslServer.java b/clients/src/main/java/org/apache/kafka/common/security/oauthbearer/internals/OAuthBearerSaslServer.java index 5d1f224883bde..aacc8fa3cbba1 100644 --- a/clients/src/main/java/org/apache/kafka/common/security/oauthbearer/internals/OAuthBearerSaslServer.java +++ b/clients/src/main/java/org/apache/kafka/common/security/oauthbearer/internals/OAuthBearerSaslServer.java @@ -21,8 +21,6 @@ import java.util.Arrays; import java.util.Map; import java.util.Objects; -import java.util.regex.Matcher; -import java.util.regex.Pattern; import javax.security.auth.callback.Callback; import javax.security.auth.callback.CallbackHandler; @@ -48,13 +46,9 @@ * for example). */ public class OAuthBearerSaslServer implements SaslServer { - private static final String INVALID_OAUTHBEARER_CLIENT_FIRST_MESSAGE = "Invalid OAUTHBEARER client first message"; private static final Logger log = LoggerFactory.getLogger(OAuthBearerSaslServer.class); private static final String NEGOTIATED_PROPERTY_KEY_TOKEN = OAuthBearerLoginModule.OAUTHBEARER_MECHANISM + ".token"; private static final String INTERNAL_ERROR_ON_SERVER = "Authentication could not be performed due to an internal error on the server"; - private static final String SASLNAME = "(?:[\\x01-\\x7F&&[^=,]]|=2C|=3D)+"; - private static final Pattern CLIENT_INITIAL_RESPONSE_PATTERN = Pattern.compile( - String.format("n,(a=(?%s))?,auth=(?[\\w]+)[ ]+(?[-_\\.a-zA-Z0-9]+)", SASLNAME)); private final AuthenticateCallbackHandler callbackHandler; @@ -90,24 +84,14 @@ public byte[] evaluateResponse(byte[] response) throws SaslException, SaslAuthen throw new SaslAuthenticationException(errorMessage); } errorMessage = null; - String responseMsg = new String(response, StandardCharsets.UTF_8); - Matcher matcher = CLIENT_INITIAL_RESPONSE_PATTERN.matcher(responseMsg); - if (!matcher.matches()) { - if (log.isDebugEnabled()) - log.debug(INVALID_OAUTHBEARER_CLIENT_FIRST_MESSAGE); - throw new SaslException(INVALID_OAUTHBEARER_CLIENT_FIRST_MESSAGE); - } - String authzid = matcher.group("authzid"); - String authorizationId = authzid != null ? authzid : ""; - if (!"bearer".equalsIgnoreCase(matcher.group("scheme"))) { - String msg = String.format("Invalid scheme in OAUTHBEARER client first message: %s", - matcher.group("scheme")); - if (log.isDebugEnabled()) - log.debug(msg); - throw new SaslException(msg); + OAuthBearerClientInitialResponse clientResponse; + try { + clientResponse = new OAuthBearerClientInitialResponse(response); + } catch (SaslException e) { + log.debug(e.getMessage()); + throw e; } - String tokenValue = matcher.group("token"); - return process(tokenValue, authorizationId); + return process(clientResponse.tokenValue(), clientResponse.authorizationId()); } @Override diff --git a/clients/src/main/java/org/apache/kafka/common/security/scram/internals/ScramExtensions.java b/clients/src/main/java/org/apache/kafka/common/security/scram/internals/ScramExtensions.java index cbfca13dbf20e..5028329feb1cb 100644 --- a/clients/src/main/java/org/apache/kafka/common/security/scram/internals/ScramExtensions.java +++ b/clients/src/main/java/org/apache/kafka/common/security/scram/internals/ScramExtensions.java @@ -17,9 +17,9 @@ package org.apache.kafka.common.security.scram.internals; import org.apache.kafka.common.security.scram.ScramLoginModule; +import org.apache.kafka.common.utils.Utils; import java.util.Collections; -import java.util.HashMap; import java.util.Map; import java.util.Set; @@ -31,7 +31,7 @@ public ScramExtensions() { } public ScramExtensions(String extensions) { - this(stringToMap(extensions)); + this(Utils.parseMap(extensions, "=", ",")); } public ScramExtensions(Map extensionMap) { @@ -52,29 +52,6 @@ public boolean tokenAuthenticated() { @Override public String toString() { - return mapToString(extensionMap); - } - - private static Map stringToMap(String extensions) { - Map extensionMap = new HashMap<>(); - - if (!extensions.isEmpty()) { - String[] attrvals = extensions.split(","); - for (String attrval : attrvals) { - String[] array = attrval.split("=", 2); - extensionMap.put(array[0], array[1]); - } - } - return extensionMap; - } - - private static String mapToString(Map extensionMap) { - StringBuilder builder = new StringBuilder(); - for (Map.Entry entry : extensionMap.entrySet()) { - builder.append(entry.getKey()); - builder.append('='); - builder.append(entry.getValue()); - } - return builder.toString(); + return Utils.mkString(extensionMap, "", "", "=", ","); } } \ No newline at end of file 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 index ebe87ba3e2797..dba9727326071 100755 --- a/clients/src/main/java/org/apache/kafka/common/utils/Utils.java +++ b/clients/src/main/java/org/apache/kafka/common/utils/Utils.java @@ -57,6 +57,7 @@ import java.util.Objects; import java.util.Properties; import java.util.Set; +import java.util.concurrent.atomic.AtomicReference; import java.util.regex.Matcher; import java.util.regex.Pattern; @@ -111,6 +112,17 @@ public static String utf8(ByteBuffer buffer, int length) { return utf8(buffer, 0, length); } + /** + * Read a UTF8 string from the current position till the end of a byte buffer. The position of the byte buffer is + * not affected by this method. + * + * @param buffer The buffer to read from + * @return The UTF8 string + */ + public static String utf8(ByteBuffer buffer) { + return utf8(buffer, buffer.remaining()); + } + /** * Read a UTF8 string from a byte buffer at a given offset. Note that the position of the byte buffer * is not affected by this method. @@ -501,6 +513,19 @@ public static String mkString(Map map, String begin, String end, return bld.toString(); } + public static Map parseMap(String mapStr, String keyValueSeparator, String elementSeparator) { + Map map = new HashMap<>(); + + if (!mapStr.isEmpty()) { + String[] attrvals = mapStr.split(elementSeparator); + for (String attrval : attrvals) { + String[] array = attrval.split(keyValueSeparator, 2); + map.put(array[0], array[1]); + } + } + return map; + } + /** * Read a properties file from the given path * @param filename The path of the file to read @@ -805,6 +830,18 @@ public static void closeQuietly(AutoCloseable closeable, String name) { } } + public static void closeQuietly(AutoCloseable closeable, String name, AtomicReference firstException) { + if (closeable != null) { + try { + closeable.close(); + } catch (Throwable t) { + firstException.compareAndSet(null, t); + log.error("Failed to close {} with type {}", name, closeable.getClass().getName(), t); + } + } + } + + /** * 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 diff --git a/clients/src/main/resources/META-INF/services/org.apache.kafka.common.config.provider.ConfigProvider b/clients/src/main/resources/META-INF/services/org.apache.kafka.common.config.provider.ConfigProvider new file mode 100644 index 0000000000000..146de7abe4442 --- /dev/null +++ b/clients/src/main/resources/META-INF/services/org.apache.kafka.common.config.provider.ConfigProvider @@ -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. + +org.apache.kafka.common.config.provider.FileConfigProvider diff --git a/clients/src/test/java/org/apache/kafka/clients/InFlightRequestsTest.java b/clients/src/test/java/org/apache/kafka/clients/InFlightRequestsTest.java index 600e5dc9053d4..c7b9eb903c8d4 100644 --- a/clients/src/test/java/org/apache/kafka/clients/InFlightRequestsTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/InFlightRequestsTest.java @@ -19,10 +19,14 @@ import org.apache.kafka.common.protocol.ApiKeys; import org.apache.kafka.common.requests.RequestHeader; +import org.apache.kafka.common.utils.MockTime; +import org.apache.kafka.common.utils.Time; import org.apache.kafka.test.TestUtils; import org.junit.Before; import org.junit.Test; +import java.util.Arrays; +import java.util.Collections; import java.util.List; import static org.junit.Assert.assertEquals; @@ -64,6 +68,24 @@ public void testClearAll() { assertEquals(correlationId2, clearedRequests.get(1).header.correlationId()); } + @Test + public void testTimedOutNodes() { + Time time = new MockTime(); + + addRequest("A", time.milliseconds(), 50); + addRequest("B", time.milliseconds(), 200); + addRequest("B", time.milliseconds(), 100); + + time.sleep(50); + assertEquals(Collections.emptyList(), inFlightRequests.nodesWithTimedOutRequests(time.milliseconds())); + + time.sleep(25); + assertEquals(Collections.singletonList("A"), inFlightRequests.nodesWithTimedOutRequests(time.milliseconds())); + + time.sleep(50); + assertEquals(Arrays.asList("A", "B"), inFlightRequests.nodesWithTimedOutRequests(time.milliseconds())); + } + @Test public void testCompleteNext() { int correlationId1 = addRequest(dest); @@ -88,12 +110,16 @@ public void testCompleteLastSentThrowsIfNoInFlights() { } private int addRequest(String destination) { + return addRequest(destination, 0, 10000); + } + + private int addRequest(String destination, long sendTimeMs, int requestTimeoutMs) { int correlationId = this.correlationId; this.correlationId += 1; RequestHeader requestHeader = new RequestHeader(ApiKeys.METADATA, (short) 0, "clientId", correlationId); - NetworkClient.InFlightRequest ifr = new NetworkClient.InFlightRequest(requestHeader, 0, - destination, null, false, false, null, null, 0); + NetworkClient.InFlightRequest ifr = new NetworkClient.InFlightRequest(requestHeader, requestTimeoutMs, 0, + destination, null, false, false, null, null, sendTimeMs); inFlightRequests.add(ifr); return correlationId; } diff --git a/clients/src/test/java/org/apache/kafka/clients/MetadataTest.java b/clients/src/test/java/org/apache/kafka/clients/MetadataTest.java index 1188af776aaa4..969921eceebac 100644 --- a/clients/src/test/java/org/apache/kafka/clients/MetadataTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/MetadataTest.java @@ -25,6 +25,7 @@ import java.util.concurrent.atomic.AtomicReference; import org.apache.kafka.common.Cluster; +import org.apache.kafka.common.KafkaException; import org.apache.kafka.common.internals.ClusterResourceListeners; import org.apache.kafka.common.Node; import org.apache.kafka.common.PartitionInfo; @@ -46,7 +47,7 @@ public class MetadataTest { private long refreshBackoffMs = 100; private long metadataExpireMs = 1000; private Metadata metadata = new Metadata(refreshBackoffMs, metadataExpireMs, true); - private AtomicReference backgroundError = new AtomicReference<>(); + private AtomicReference backgroundError = new AtomicReference<>(); @After public void tearDown() { @@ -83,6 +84,30 @@ public void testMetadata() throws Exception { assertTrue("Update needed due to stale metadata.", metadata.timeToNextUpdate(time) == 0); } + @Test + public void testMetadataAwaitAfterClose() throws InterruptedException { + long time = 0; + metadata.update(Cluster.empty(), Collections.emptySet(), 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"; + metadata.close(); + Thread t1 = asyncFetch(topic, 500); + t1.join(); + assertTrue(backgroundError.get().getClass() == KafkaException.class); + assertTrue(backgroundError.get().toString().contains("Requested metadata update after close")); + clearBackgroundError(); + } + + @Test(expected = IllegalStateException.class) + public void testMetadataUpdateAfterClose() { + metadata.close(); + metadata.update(Cluster.empty(), Collections.emptySet(), 1000); + } + private static void checkTimeToNextUpdate(long refreshBackoffMs, long metadataExpireMs) { long now = 10000; @@ -409,15 +434,18 @@ public void testNonExpiringMetadata() throws Exception { assertTrue("Unused topic expired when expiry disabled", metadata.containsTopic("topic4")); } + private void clearBackgroundError() { + backgroundError.set(null); + } + private Thread asyncFetch(final String topic, final long maxWaitMs) { Thread thread = new Thread() { public void run() { - while (metadata.fetch().partitionsForTopic(topic).isEmpty()) { - try { + try { + while (metadata.fetch().partitionsForTopic(topic).isEmpty()) metadata.awaitUpdate(metadata.requestUpdate(), maxWaitMs); - } catch (Exception e) { - backgroundError.set(e.toString()); - } + } catch (Exception e) { + backgroundError.set(e); } } }; 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 2a1e21375b96d..4bd5e54ed2d1d 100644 --- a/clients/src/test/java/org/apache/kafka/clients/MockClient.java +++ b/clients/src/test/java/org/apache/kafka/clients/MockClient.java @@ -83,6 +83,8 @@ public FutureResponse(Node node, private final TransientSet blackedOut; // Nodes which will always fail to connect, but can be chosen by leastLoadedNode private final TransientSet unreachable; + // Nodes which have a delay before ultimately succeeding to connect + private final TransientSet delayedReady; private final Map pendingAuthenticationErrors = new HashMap<>(); private final Map authenticationErrors = new HashMap<>(); @@ -94,6 +96,7 @@ public FutureResponse(Node node, private final Queue metadataUpdates = new ConcurrentLinkedDeque<>(); private volatile NodeApiVersions nodeApiVersions = NodeApiVersions.create(); private volatile int numBlockingWakeups = 0; + private volatile boolean active = true; public MockClient(Time time) { this(time, null); @@ -105,6 +108,7 @@ public MockClient(Time time, Metadata metadata) { this.unavailableTopics = Collections.emptySet(); this.blackedOut = new TransientSet<>(time); this.unreachable = new TransientSet<>(time); + this.delayedReady = new TransientSet<>(time); } @Override @@ -122,6 +126,9 @@ public boolean ready(Node node, long now) { return false; } + if (delayedReady.contains(node, now)) + return false; + ready.add(node.idString()); return true; } @@ -145,6 +152,10 @@ public void setUnreachable(Node node, long durationMs) { unreachable.add(node, durationMs); } + public void delayReady(Node node, long durationMs) { + delayedReady.add(node, durationMs); + } + public void authenticationFailed(Node node, long blackoutMs) { pendingAuthenticationErrors.remove(node); authenticationErrors.put(node, (AuthenticationException) Errors.SASL_AUTHENTICATION_FAILED.exception()); @@ -267,8 +278,7 @@ private synchronized void maybeAwaitWakeup() { @Override public List poll(long timeoutMs, long now) { maybeAwaitWakeup(); - - List copy = new ArrayList<>(this.responses); + checkTimeoutOfPendingRequests(now); if (metadata != null && metadata.updateRequested()) { MetadataUpdate metadataUpdate = metadataUpdates.poll(); @@ -288,14 +298,29 @@ public List poll(long timeoutMs, long now) { } } + List copy = new ArrayList<>(); ClientResponse response; while ((response = this.responses.poll()) != null) { response.onComplete(); + copy.add(response); } return copy; } + private long elapsedTimeMs(long currentTimeMs, long startTimeMs) { + return Math.max(0, currentTimeMs - startTimeMs); + } + + private void checkTimeoutOfPendingRequests(long nowMs) { + ClientRequest request = requests.peek(); + while (request != null && elapsedTimeMs(nowMs, request.createdTimeMs()) > request.requestTimeoutMs()) { + disconnect(request.destination()); + requests.poll(); + request = requests.peek(); + } + } + public Queue requests() { return this.requests; } @@ -467,7 +492,7 @@ public boolean hasInFlightRequests() { } public boolean hasPendingResponses() { - return !responses.isEmpty(); + return !responses.isEmpty() || !futureResponses.isEmpty(); } @Override @@ -493,18 +518,34 @@ public boolean hasReadyNodes(long now) { @Override public ClientRequest newClientRequest(String nodeId, AbstractRequest.Builder requestBuilder, long createdTimeMs, boolean expectResponse) { - return newClientRequest(nodeId, requestBuilder, createdTimeMs, expectResponse, null); + return newClientRequest(nodeId, requestBuilder, createdTimeMs, expectResponse, 5000, null); } @Override - public ClientRequest newClientRequest(String nodeId, AbstractRequest.Builder requestBuilder, long createdTimeMs, - boolean expectResponse, RequestCompletionHandler callback) { + public ClientRequest newClientRequest(String nodeId, + AbstractRequest.Builder requestBuilder, + long createdTimeMs, + boolean expectResponse, + int requestTimeoutMs, + RequestCompletionHandler callback) { return new ClientRequest(nodeId, requestBuilder, correlation++, "mockClientId", createdTimeMs, - expectResponse, callback); + expectResponse, requestTimeoutMs, callback); + } + + @Override + public void initiateClose() { + close(); + } + + @Override + public boolean active() { + return active; } @Override public void close() { + active = false; + metadata.close(); } @Override 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 f83226c42ac02..2876570b2bfab 100644 --- a/clients/src/test/java/org/apache/kafka/clients/NetworkClientTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/NetworkClientTest.java @@ -47,11 +47,12 @@ import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertFalse; import static org.junit.Assert.assertNotEquals; +import static org.junit.Assert.assertNull; import static org.junit.Assert.assertTrue; public class NetworkClientTest { - protected final int requestTimeoutMs = 1000; + protected final int defaultRequestTimeoutMs = 1000; protected final MockTime time = new MockTime(); protected final MockSelector selector = new MockSelector(time); protected final Metadata metadata = new Metadata(0, Long.MAX_VALUE, true); @@ -69,19 +70,19 @@ public class NetworkClientTest { private NetworkClient createNetworkClient(long reconnectBackoffMaxMs) { return new NetworkClient(selector, metadata, "mock", Integer.MAX_VALUE, reconnectBackoffMsTest, reconnectBackoffMaxMs, 64 * 1024, 64 * 1024, - requestTimeoutMs, time, true, new ApiVersions(), new LogContext()); + defaultRequestTimeoutMs, time, true, new ApiVersions(), new LogContext()); } private NetworkClient createNetworkClientWithStaticNodes() { return new NetworkClient(selector, new ManualMetadataUpdater(Arrays.asList(node)), - "mock-static", Integer.MAX_VALUE, 0, 0, 64 * 1024, 64 * 1024, requestTimeoutMs, + "mock-static", Integer.MAX_VALUE, 0, 0, 64 * 1024, 64 * 1024, defaultRequestTimeoutMs, time, true, new ApiVersions(), new LogContext()); } private NetworkClient createNetworkClientWithNoVersionDiscovery() { return new NetworkClient(selector, metadata, "mock", Integer.MAX_VALUE, reconnectBackoffMsTest, reconnectBackoffMaxMsTest, - 64 * 1024, 64 * 1024, requestTimeoutMs, time, false, new ApiVersions(), new LogContext()); + 64 * 1024, 64 * 1024, defaultRequestTimeoutMs, time, false, new ApiVersions(), new LogContext()); } @Before @@ -140,10 +141,10 @@ public void testClose() { private void checkSimpleRequestResponse(NetworkClient networkClient) { awaitReady(networkClient, node); // has to be before creating any request, as it may send ApiVersionsRequest and its response is mocked with correlation id 0 ProduceRequest.Builder builder = ProduceRequest.Builder.forCurrentMagic((short) 1, 1000, - Collections.emptyMap()); + Collections.emptyMap()); TestCallbackHandler handler = new TestCallbackHandler(); ClientRequest request = networkClient.newClientRequest( - node.idString(), builder, time.milliseconds(), true, handler); + node.idString(), builder, time.milliseconds(), true, defaultRequestTimeoutMs, handler); networkClient.send(request, time.milliseconds()); networkClient.poll(1, time.milliseconds()); assertEquals(1, networkClient.inFlightRequestCount()); @@ -184,16 +185,30 @@ private void awaitReady(NetworkClient client, Node node) { public void testRequestTimeout() { awaitReady(client, node); // has to be before creating any request, as it may send ApiVersionsRequest and its response is mocked with correlation id 0 ProduceRequest.Builder builder = ProduceRequest.Builder.forCurrentMagic((short) 1, - 1000, Collections.emptyMap()); + 1000, Collections.emptyMap()); TestCallbackHandler handler = new TestCallbackHandler(); - long now = time.milliseconds(); - ClientRequest request = client.newClientRequest( - node.idString(), builder, now, true, handler); - client.send(request, now); + int requestTimeoutMs = defaultRequestTimeoutMs + 5000; + ClientRequest request = client.newClientRequest(node.idString(), builder, time.milliseconds(), true, + requestTimeoutMs, handler); + assertEquals(requestTimeoutMs, request.requestTimeoutMs()); + testRequestTimeout(request); + } + + @Test + public void testDefaultRequestTimeout() { + awaitReady(client, node); // has to be before creating any request, as it may send ApiVersionsRequest and its response is mocked with correlation id 0 + ProduceRequest.Builder builder = ProduceRequest.Builder.forCurrentMagic((short) 1, + 1000, Collections.emptyMap()); + ClientRequest request = client.newClientRequest(node.idString(), builder, time.milliseconds(), true); + assertEquals(defaultRequestTimeoutMs, request.requestTimeoutMs()); + testRequestTimeout(request); + } - // sleeping to make sure that the time since last send is greater than requestTimeOut - time.sleep(3000); - List responses = client.poll(3000, time.milliseconds()); + private void testRequestTimeout(ClientRequest request) { + client.send(request, time.milliseconds()); + + time.sleep(request.requestTimeoutMs() + 1); + List responses = client.poll(0, time.milliseconds()); assertEquals(1, responses.size()); ClientResponse clientResponse = responses.get(0); @@ -206,9 +221,10 @@ public void testConnectionThrottling() { // Instrument the test to return a response with a 100ms throttle delay. awaitReady(client, node); ProduceRequest.Builder builder = ProduceRequest.Builder.forCurrentMagic((short) 1, 1000, - Collections.emptyMap()); + Collections.emptyMap()); TestCallbackHandler handler = new TestCallbackHandler(); - ClientRequest request = client.newClientRequest(node.idString(), builder, time.milliseconds(), true, handler); + ClientRequest request = client.newClientRequest(node.idString(), builder, time.milliseconds(), true, + defaultRequestTimeoutMs, handler); client.send(request, time.milliseconds()); client.poll(1, time.milliseconds()); ResponseHeader respHeader = new ResponseHeader(request.correlationId()); @@ -222,7 +238,7 @@ public void testConnectionThrottling() { resp.writeTo(buffer); buffer.flip(); selector.completeReceive(new NetworkReceive(node.idString(), buffer)); - List responses = client.poll(1, time.milliseconds()); + client.poll(1, time.milliseconds()); // The connection is not ready due to throttling. assertFalse(client.ready(node, time.milliseconds())); @@ -264,9 +280,10 @@ public void testThrottlingNotEnabledForConnectionToOlderBroker() { selector.clear(); ProduceRequest.Builder builder = ProduceRequest.Builder.forCurrentMagic((short) 1, 1000, - Collections.emptyMap()); + Collections.emptyMap()); TestCallbackHandler handler = new TestCallbackHandler(); - ClientRequest request = client.newClientRequest(node.idString(), builder, time.milliseconds(), true, handler); + ClientRequest request = client.newClientRequest(node.idString(), builder, time.milliseconds(), true, + defaultRequestTimeoutMs, handler); client.send(request, time.milliseconds()); client.poll(1, time.milliseconds()); ResponseHeader respHeader = new ResponseHeader(request.correlationId()); @@ -280,7 +297,7 @@ public void testThrottlingNotEnabledForConnectionToOlderBroker() { resp.writeTo(buffer); buffer.flip(); selector.completeReceive(new NetworkReceive(node.idString(), buffer)); - List responses = client.poll(1, time.milliseconds()); + client.poll(1, time.milliseconds()); // Since client-side throttling is disabled, the connection is ready even though the response indicated a // throttle delay. @@ -308,7 +325,7 @@ public void testLeastLoadedNode() { client.poll(1, time.milliseconds()); assertFalse("After we forced the disconnection the client is no longer ready.", client.ready(node, time.milliseconds())); leastNode = client.leastLoadedNode(time.milliseconds()); - assertEquals("There should be NO leastloadednode", leastNode, null); + assertNull("There should be NO leastloadednode", leastNode); } @Test @@ -334,7 +351,7 @@ public void testConnectionDelayDisconnectedWithNoExponentialBackoff() { awaitReady(clientWithNoExponentialBackoff, node); selector.serverDisconnect(node.idString()); - clientWithNoExponentialBackoff.poll(requestTimeoutMs, time.milliseconds()); + clientWithNoExponentialBackoff.poll(defaultRequestTimeoutMs, time.milliseconds()); long delay = clientWithNoExponentialBackoff.connectionDelay(node, time.milliseconds()); assertEquals(reconnectBackoffMsTest, delay); @@ -346,7 +363,7 @@ public void testConnectionDelayDisconnectedWithNoExponentialBackoff() { // Start connecting and disconnect before the connection is established client.ready(node, time.milliseconds()); selector.serverDisconnect(node.idString()); - client.poll(requestTimeoutMs, time.milliseconds()); + client.poll(defaultRequestTimeoutMs, time.milliseconds()); // Second attempt should have the same behaviour as exponential backoff is disabled assertEquals(reconnectBackoffMsTest, delay); @@ -376,7 +393,7 @@ public void testConnectionDelayDisconnected() { // First disconnection selector.serverDisconnect(node.idString()); - client.poll(requestTimeoutMs, time.milliseconds()); + client.poll(defaultRequestTimeoutMs, time.milliseconds()); long delay = client.connectionDelay(node, time.milliseconds()); long expectedDelay = reconnectBackoffMsTest; double jitter = 0.3; @@ -389,7 +406,7 @@ public void testConnectionDelayDisconnected() { // Start connecting and disconnect before the connection is established client.ready(node, time.milliseconds()); selector.serverDisconnect(node.idString()); - client.poll(requestTimeoutMs, time.milliseconds()); + client.poll(defaultRequestTimeoutMs, time.milliseconds()); // Second attempt should take twice as long with twice the jitter expectedDelay = Math.round(delay * 2); @@ -408,13 +425,13 @@ public void testDisconnectDuringUserMetadataRequest() { long now = time.milliseconds(); ClientRequest request = client.newClientRequest(node.idString(), builder, now, true); client.send(request, now); - client.poll(requestTimeoutMs, now); + client.poll(defaultRequestTimeoutMs, now); assertEquals(1, client.inFlightRequestCount(node.idString())); assertTrue(client.hasInFlightRequests(node.idString())); assertTrue(client.hasInFlightRequests()); selector.close(node.idString()); - List responses = client.poll(requestTimeoutMs, time.milliseconds()); + List responses = client.poll(defaultRequestTimeoutMs, time.milliseconds()); assertEquals(1, responses.size()); assertTrue(responses.iterator().next().wasDisconnected()); } @@ -442,7 +459,7 @@ public void testClientDisconnectAfterInternalApiVersionRequest() throws Exceptio } @Test - public void testDisconnectWithMultipleInFlights() throws Exception { + public void testDisconnectWithMultipleInFlights() { NetworkClient client = this.clientWithNoVersionDiscovery; awaitReady(client, node); assertTrue("Expected NetworkClient to be ready to send to node " + node.idString(), @@ -459,11 +476,11 @@ public void onComplete(ClientResponse response) { } }; - ClientRequest request1 = client.newClientRequest(node.idString(), builder, now, true, callback); + ClientRequest request1 = client.newClientRequest(node.idString(), builder, now, true, defaultRequestTimeoutMs, callback); client.send(request1, now); client.poll(0, now); - ClientRequest request2 = client.newClientRequest(node.idString(), builder, now, true, callback); + ClientRequest request2 = client.newClientRequest(node.idString(), builder, now, true, defaultRequestTimeoutMs, callback); client.send(request2, now); client.poll(0, now); diff --git a/clients/src/test/java/org/apache/kafka/clients/admin/AdminClientUnitTestEnv.java b/clients/src/test/java/org/apache/kafka/clients/admin/AdminClientUnitTestEnv.java index 3cd807d93dca4..5ec4d18811ff3 100644 --- a/clients/src/test/java/org/apache/kafka/clients/admin/AdminClientUnitTestEnv.java +++ b/clients/src/test/java/org/apache/kafka/clients/admin/AdminClientUnitTestEnv.java @@ -33,6 +33,10 @@ * To use in a test, create an instance and prepare its {@link #kafkaClient() MockClient} with the expected responses * for the {@link AdminClient}. Then, use the {@link #adminClient() AdminClient} in the test, which will then use the MockClient * and receive the responses you provided. + * + * Since {@link #kafkaClient() MockClient} is not thread-safe, + * users should be wary of calling its methods after the {@link #adminClient() AdminClient} is instantiated. + * *

* When finished, be sure to {@link #close() close} the environment object. */ @@ -109,4 +113,8 @@ private static Map newStrMap(String... vals) { } return map; } + + public static String kafkaAdminClientNetworkThreadPrefix() { + return KafkaAdminClient.NETWORK_THREAD_PREFIX; + } } diff --git a/clients/src/test/java/org/apache/kafka/clients/admin/KafkaAdminClientTest.java b/clients/src/test/java/org/apache/kafka/clients/admin/KafkaAdminClientTest.java index 3a300dbfb1bfa..8d2b92ffeba22 100644 --- a/clients/src/test/java/org/apache/kafka/clients/admin/KafkaAdminClientTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/admin/KafkaAdminClientTest.java @@ -36,7 +36,7 @@ import org.apache.kafka.common.acl.AclPermissionType; import org.apache.kafka.common.config.ConfigResource; import org.apache.kafka.common.errors.AuthenticationException; -import org.apache.kafka.common.errors.CoordinatorNotAvailableException; +import org.apache.kafka.common.errors.GroupAuthorizationException; import org.apache.kafka.common.errors.InvalidTopicException; import org.apache.kafka.common.errors.LeaderNotAvailableException; import org.apache.kafka.common.errors.NotLeaderForPartitionException; @@ -44,6 +44,7 @@ import org.apache.kafka.common.errors.SaslAuthenticationException; import org.apache.kafka.common.errors.SecurityDisabledException; import org.apache.kafka.common.errors.TimeoutException; +import org.apache.kafka.common.errors.UnknownServerException; import org.apache.kafka.common.errors.UnknownTopicOrPartitionException; import org.apache.kafka.common.protocol.Errors; import org.apache.kafka.common.requests.ApiError; @@ -65,9 +66,9 @@ import org.apache.kafka.common.requests.MetadataRequest; import org.apache.kafka.common.requests.MetadataResponse; import org.apache.kafka.common.requests.OffsetFetchResponse; -import org.apache.kafka.common.resource.Resource; -import org.apache.kafka.common.resource.ResourceFilter; -import org.apache.kafka.common.resource.ResourceNameType; +import org.apache.kafka.common.resource.PatternType; +import org.apache.kafka.common.resource.ResourcePattern; +import org.apache.kafka.common.resource.ResourcePatternFilter; import org.apache.kafka.common.resource.ResourceType; import org.apache.kafka.common.utils.MockTime; import org.apache.kafka.common.utils.Time; @@ -98,8 +99,6 @@ import static java.util.Arrays.asList; import static java.util.Collections.singletonList; -import static org.apache.kafka.common.requests.ResourceType.BROKER; -import static org.apache.kafka.common.requests.ResourceType.TOPIC; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertNotNull; import static org.junit.Assert.assertNull; @@ -537,13 +536,13 @@ private void callAdminClientApisAndExpectAnAuthenticationError(AdminClientUnitTe } } - private static final AclBinding ACL1 = new AclBinding(new Resource(ResourceType.TOPIC, "mytopic3", ResourceNameType.LITERAL), + private static final AclBinding ACL1 = new AclBinding(new ResourcePattern(ResourceType.TOPIC, "mytopic3", PatternType.LITERAL), new AccessControlEntry("User:ANONYMOUS", "*", AclOperation.DESCRIBE, AclPermissionType.ALLOW)); - private static final AclBinding ACL2 = new AclBinding(new Resource(ResourceType.TOPIC, "mytopic4", ResourceNameType.LITERAL), + private static final AclBinding ACL2 = new AclBinding(new ResourcePattern(ResourceType.TOPIC, "mytopic4", PatternType.LITERAL), new AccessControlEntry("User:ANONYMOUS", "*", AclOperation.DESCRIBE, AclPermissionType.DENY)); - private static final AclBindingFilter FILTER1 = new AclBindingFilter(new ResourceFilter(ResourceType.ANY, null, ResourceNameType.LITERAL), + private static final AclBindingFilter FILTER1 = new AclBindingFilter(new ResourcePatternFilter(ResourceType.ANY, null, PatternType.LITERAL), new AccessControlEntryFilter("User:ANONYMOUS", null, AclOperation.ANY, AclPermissionType.ANY)); - private static final AclBindingFilter FILTER2 = new AclBindingFilter(new ResourceFilter(ResourceType.ANY, null, ResourceNameType.LITERAL), + private static final AclBindingFilter FILTER2 = new AclBindingFilter(new ResourcePatternFilter(ResourceType.ANY, null, PatternType.LITERAL), new AccessControlEntryFilter("User:bob", null, AclOperation.ANY, AclPermissionType.ANY)); @Test @@ -680,9 +679,9 @@ public boolean conditionMet() { // The next request should succeed. time.sleep(5000); env.kafkaClient().prepareResponse(new DescribeConfigsResponse(0, - Collections.singletonMap(new org.apache.kafka.common.requests.Resource(TOPIC, "foo"), + Collections.singletonMap(new ConfigResource(ConfigResource.Type.TOPIC, "foo"), new DescribeConfigsResponse.Config(ApiError.NONE, - Collections.emptySet())))); + Collections.emptySet())))); DescribeConfigsResult result2 = env.adminClient().describeConfigs(Collections.singleton( new ConfigResource(ConfigResource.Type.TOPIC, "foo"))); time.sleep(5000); @@ -696,9 +695,8 @@ public void testDescribeConfigs() throws Exception { env.kafkaClient().setNodeApiVersions(NodeApiVersions.create()); env.kafkaClient().setNode(env.cluster().controller()); env.kafkaClient().prepareResponse(new DescribeConfigsResponse(0, - Collections.singletonMap(new org.apache.kafka.common.requests.Resource(BROKER, "0"), - new DescribeConfigsResponse.Config(ApiError.NONE, - Collections.emptySet())))); + Collections.singletonMap(new ConfigResource(ConfigResource.Type.BROKER, "0"), + new DescribeConfigsResponse.Config(ApiError.NONE, Collections.emptySet())))); DescribeConfigsResult result2 = env.adminClient().describeConfigs(Collections.singleton( new ConfigResource(ConfigResource.Type.BROKER, "0"))); result2.all().get(); @@ -850,9 +848,11 @@ public void testListConsumerGroups() throws Exception { Node node0 = new Node(0, "localhost", 8121); Node node1 = new Node(1, "localhost", 8122); Node node2 = new Node(2, "localhost", 8123); + Node node3 = new Node(3, "localhost", 8124); nodes.put(0, node0); nodes.put(1, node1); nodes.put(2, node2); + nodes.put(3, node3); final Cluster cluster = new Cluster( "mockClusterId", @@ -861,7 +861,7 @@ public void testListConsumerGroups() throws Exception { Collections.emptySet(), Collections.emptySet(), nodes.get(0)); - try (AdminClientUnitTestEnv env = new AdminClientUnitTestEnv(cluster)) { + try (AdminClientUnitTestEnv env = new AdminClientUnitTestEnv(cluster, AdminClientConfig.RETRIES_CONFIG, "2")) { env.kafkaClient().setNodeApiVersions(NodeApiVersions.create()); env.kafkaClient().setNode(env.cluster().controller()); @@ -889,13 +889,19 @@ public void testListConsumerGroups() throws Exception { )), node0); + // handle retriable errors env.kafkaClient().prepareResponseFrom( new ListGroupsResponse( - Errors.COORDINATOR_NOT_AVAILABLE, + Errors.COORDINATOR_NOT_AVAILABLE, + Collections.emptyList() + ), + node1); + env.kafkaClient().prepareResponseFrom( + new ListGroupsResponse( + Errors.COORDINATOR_LOAD_IN_PROGRESS, Collections.emptyList() ), node1); - env.kafkaClient().prepareResponseFrom( new ListGroupsResponse( Errors.NONE, @@ -903,15 +909,37 @@ public void testListConsumerGroups() throws Exception { new ListGroupsResponse.Group("group-2", ConsumerProtocol.PROTOCOL_TYPE), new ListGroupsResponse.Group("group-connect-2", "connector") )), + node1); + + env.kafkaClient().prepareResponseFrom( + new ListGroupsResponse( + Errors.NONE, + asList( + new ListGroupsResponse.Group("group-3", ConsumerProtocol.PROTOCOL_TYPE), + new ListGroupsResponse.Group("group-connect-3", "connector") + )), node2); + // fatal error + env.kafkaClient().prepareResponseFrom( + new ListGroupsResponse( + Errors.UNKNOWN_SERVER_ERROR, + Collections.emptyList()), + node3); + + final ListConsumerGroupsResult result = env.adminClient().listConsumerGroups(); - assertFutureError(result.all(), CoordinatorNotAvailableException.class); + assertFutureError(result.all(), UnknownServerException.class); + Collection listings = result.valid().get(); - assertEquals(2, listings.size()); + assertEquals(3, listings.size()); + + Set groupIds = new HashSet<>(); for (ConsumerGroupListing listing : listings) { - assertTrue(listing.groupId().equals("group-1") || listing.groupId().equals("group-2")); + groupIds.add(listing.groupId()); } + + assertEquals(Utils.mkSet("group-1", "group-2", "group-3"), groupIds); assertEquals(1, result.errors().get().size()); } } @@ -970,9 +998,34 @@ public void testDescribeConsumerGroups() throws Exception { env.kafkaClient().setNodeApiVersions(NodeApiVersions.create()); env.kafkaClient().setNode(env.cluster().controller()); + //Retriable FindCoordinatorResponse errors should be retried + env.kafkaClient().prepareResponse(new FindCoordinatorResponse(Errors.COORDINATOR_NOT_AVAILABLE, Node.noNode())); + env.kafkaClient().prepareResponse(new FindCoordinatorResponse(Errors.COORDINATOR_LOAD_IN_PROGRESS, Node.noNode())); env.kafkaClient().prepareResponse(new FindCoordinatorResponse(Errors.NONE, env.cluster().controller())); - final Map groupMetadataMap = new HashMap<>(); + Map groupMetadataMap = new HashMap<>(); + + //Retriable errors should be retried + groupMetadataMap.put( + "group-0", + new DescribeGroupsResponse.GroupMetadata( + Errors.COORDINATOR_NOT_AVAILABLE, + "", + ConsumerProtocol.PROTOCOL_TYPE, + "", + Collections.emptyList())); + groupMetadataMap.put( + "group-connect-0", + new DescribeGroupsResponse.GroupMetadata( + Errors.COORDINATOR_LOAD_IN_PROGRESS, + "", + "connect", + "", + Collections.emptyList())); + env.kafkaClient().prepareResponse(new DescribeGroupsResponse(groupMetadataMap)); + + groupMetadataMap = new HashMap<>(); + TopicPartition myTopicPartition0 = new TopicPartition("my_topic", 0); TopicPartition myTopicPartition1 = new TopicPartition("my_topic", 1); TopicPartition myTopicPartition2 = new TopicPartition("my_topic", 2); @@ -1033,8 +1086,15 @@ public void testDescribeConsumerGroupOffsets() throws Exception { env.kafkaClient().setNodeApiVersions(NodeApiVersions.create()); env.kafkaClient().setNode(env.cluster().controller()); + //Retriable FindCoordinatorResponse errors should be retried + env.kafkaClient().prepareResponse(new FindCoordinatorResponse(Errors.COORDINATOR_NOT_AVAILABLE, Node.noNode())); + env.kafkaClient().prepareResponse(new FindCoordinatorResponse(Errors.NONE, env.cluster().controller())); + //Retriable errors should be retried + env.kafkaClient().prepareResponse(new OffsetFetchResponse(Errors.COORDINATOR_NOT_AVAILABLE, Collections.emptyMap())); + env.kafkaClient().prepareResponse(new OffsetFetchResponse(Errors.COORDINATOR_LOAD_IN_PROGRESS, Collections.emptyMap())); + TopicPartition myTopicPartition0 = new TopicPartition("my_topic", 0); TopicPartition myTopicPartition1 = new TopicPartition("my_topic", 1); TopicPartition myTopicPartition2 = new TopicPartition("my_topic", 2); @@ -1074,16 +1134,44 @@ public void testDeleteConsumerGroups() throws Exception { env.kafkaClient().setNodeApiVersions(NodeApiVersions.create()); env.kafkaClient().setNode(env.cluster().controller()); + //Retriable FindCoordinatorResponse errors should be retried + env.kafkaClient().prepareResponse(new FindCoordinatorResponse(Errors.COORDINATOR_NOT_AVAILABLE, Node.noNode())); + env.kafkaClient().prepareResponse(new FindCoordinatorResponse(Errors.COORDINATOR_LOAD_IN_PROGRESS, Node.noNode())); + env.kafkaClient().prepareResponse(new FindCoordinatorResponse(Errors.NONE, env.cluster().controller())); - final Map response = new HashMap<>(); - response.put("group-0", Errors.NONE); - env.kafkaClient().prepareResponse(new DeleteGroupsResponse(response)); + final Map validResponse = new HashMap<>(); + validResponse.put("group-0", Errors.NONE); + env.kafkaClient().prepareResponse(new DeleteGroupsResponse(validResponse)); final DeleteConsumerGroupsResult result = env.adminClient().deleteConsumerGroups(groupIds); final KafkaFuture results = result.deletedGroups().get("group-0"); assertNull(results.get()); + + //should throw error for non-retriable errors + env.kafkaClient().prepareResponse(new FindCoordinatorResponse(Errors.GROUP_AUTHORIZATION_FAILED, Node.noNode())); + + final DeleteConsumerGroupsResult errorResult = env.adminClient().deleteConsumerGroups(groupIds); + assertFutureError(errorResult.deletedGroups().get("group-0"), GroupAuthorizationException.class); + + //Retriable errors should be retried + env.kafkaClient().prepareResponse(new FindCoordinatorResponse(Errors.NONE, env.cluster().controller())); + + final Map errorResponse1 = new HashMap<>(); + errorResponse1.put("group-0", Errors.COORDINATOR_NOT_AVAILABLE); + env.kafkaClient().prepareResponse(new DeleteGroupsResponse(errorResponse1)); + + final Map errorResponse2 = new HashMap<>(); + errorResponse2.put("group-0", Errors.COORDINATOR_LOAD_IN_PROGRESS); + env.kafkaClient().prepareResponse(new DeleteGroupsResponse(errorResponse2)); + + env.kafkaClient().prepareResponse(new DeleteGroupsResponse(validResponse)); + + final DeleteConsumerGroupsResult errorResult1 = env.adminClient().deleteConsumerGroups(groupIds); + + final KafkaFuture errorResults = errorResult1.deletedGroups().get("group-0"); + assertNull(errorResults.get()); } } diff --git a/clients/src/test/java/org/apache/kafka/clients/admin/MockAdminClient.java b/clients/src/test/java/org/apache/kafka/clients/admin/MockAdminClient.java index 2fc7048b759aa..51750720240ad 100644 --- a/clients/src/test/java/org/apache/kafka/clients/admin/MockAdminClient.java +++ b/clients/src/test/java/org/apache/kafka/clients/admin/MockAdminClient.java @@ -390,5 +390,4 @@ private final static class TopicMetadata { this.configs = configs != null ? configs : Collections.emptyMap(); } } - } 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 index 4be688422ce77..c7cfeb05bbd77 100644 --- a/clients/src/test/java/org/apache/kafka/clients/consumer/KafkaConsumerTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/consumer/KafkaConsumerTest.java @@ -26,6 +26,7 @@ import org.apache.kafka.clients.consumer.internals.ConsumerNetworkClient; import org.apache.kafka.clients.consumer.internals.ConsumerProtocol; import org.apache.kafka.clients.consumer.internals.Fetcher; +import org.apache.kafka.clients.consumer.internals.Heartbeat; import org.apache.kafka.clients.consumer.internals.PartitionAssignor; import org.apache.kafka.clients.consumer.internals.SubscriptionState; import org.apache.kafka.common.Cluster; @@ -44,10 +45,10 @@ import org.apache.kafka.common.record.MemoryRecords; import org.apache.kafka.common.record.MemoryRecordsBuilder; import org.apache.kafka.common.record.TimestampType; -import org.apache.kafka.common.requests.FetchResponse; import org.apache.kafka.common.requests.AbstractRequest; import org.apache.kafka.common.requests.AbstractResponse; import org.apache.kafka.common.requests.FetchRequest; +import org.apache.kafka.common.requests.FetchResponse; import org.apache.kafka.common.requests.FindCoordinatorResponse; import org.apache.kafka.common.requests.HeartbeatResponse; import org.apache.kafka.common.requests.IsolationLevel; @@ -71,6 +72,7 @@ import org.apache.kafka.test.MockMetricsReporter; import org.apache.kafka.test.TestCondition; import org.apache.kafka.test.TestUtils; +import org.easymock.EasyMock; import org.junit.Assert; import org.junit.Rule; import org.junit.Test; @@ -1747,7 +1749,8 @@ private KafkaConsumer newConsumer(Time time, String groupId = "mock-group"; String metricGroupPrefix = "consumer"; long retryBackoffMs = 100; - long requestTimeoutMs = 30000; + int requestTimeoutMs = 30000; + int defaultApiTimeoutMs = 30000; boolean excludeInternalTopics = true; int minBytes = 1; int maxBytes = Integer.MAX_VALUE; @@ -1761,7 +1764,7 @@ private KafkaConsumer newConsumer(Time time, Deserializer valueDeserializer = new StringDeserializer(); List assignors = singletonList(assignor); - ConsumerInterceptors interceptors = new ConsumerInterceptors<>(Collections.>emptyList()); + ConsumerInterceptors interceptors = new ConsumerInterceptors<>(Collections.emptyList()); Metrics metrics = new Metrics(); ConsumerMetrics metricsRegistry = new ConsumerMetrics(metricGroupPrefix); @@ -1770,13 +1773,15 @@ private KafkaConsumer newConsumer(Time time, LogContext loggerFactory = new LogContext(); ConsumerNetworkClient consumerClient = new ConsumerNetworkClient(loggerFactory, client, metadata, time, retryBackoffMs, requestTimeoutMs, heartbeatIntervalMs); + + Heartbeat heartbeat = new Heartbeat(sessionTimeoutMs, heartbeatIntervalMs, rebalanceTimeoutMs, retryBackoffMs); ConsumerCoordinator consumerCoordinator = new ConsumerCoordinator( loggerFactory, consumerClient, groupId, rebalanceTimeoutMs, sessionTimeoutMs, - heartbeatIntervalMs, + heartbeat, assignors, metadata, subscriptions, @@ -1810,7 +1815,7 @@ private KafkaConsumer newConsumer(Time time, requestTimeoutMs, IsolationLevel.READ_UNCOMMITTED); - return new KafkaConsumer<>( + return new KafkaConsumer( loggerFactory, clientId, consumerCoordinator, @@ -1825,6 +1830,7 @@ private KafkaConsumer newConsumer(Time time, metadata, retryBackoffMs, requestTimeoutMs, + defaultApiTimeoutMs, assignors); } @@ -1837,4 +1843,15 @@ private static class FetchInfo { this.count = count; } } + + @Test + public void testCloseWithTimeUnit() { + KafkaConsumer consumer = EasyMock.partialMockBuilder(KafkaConsumer.class) + .addMockedMethod("close", Duration.class).createMock(); + consumer.close(Duration.ofSeconds(1)); + EasyMock.expectLastCall(); + EasyMock.replay(consumer); + consumer.close(1, TimeUnit.SECONDS); + EasyMock.verify(consumer); + } } 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 index 1d01eb6d0b261..03013e6a9fa0b 100644 --- a/clients/src/test/java/org/apache/kafka/clients/consumer/MockConsumerTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/consumer/MockConsumerTest.java @@ -26,8 +26,10 @@ import java.util.HashMap; import java.util.Iterator; +import static org.hamcrest.CoreMatchers.is; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertThat; public class MockConsumerTest { @@ -84,4 +86,16 @@ public void testSimpleMockDeprecated() { assertEquals(2L, consumer.committed(new TopicPartition("test", 0)).offset()); } + @Test + public void testConsumerRecordsIsEmptyWhenReturningNoRecords() { + TopicPartition partition = new TopicPartition("test", 0); + consumer.assign(Collections.singleton(partition)); + consumer.addRecord(new ConsumerRecord("test", 0, 0, null, null)); + consumer.updateEndOffsets(Collections.singletonMap(partition, 1L)); + consumer.seekToEnd(Collections.singleton(partition)); + ConsumerRecords records = consumer.poll(Duration.ofMillis(1)); + assertThat(records.count(), is(0)); + assertThat(records.isEmpty(), is(true)); + } + } diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/AbstractCoordinatorTest.java b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/AbstractCoordinatorTest.java index 32aae442388c5..f88e72505a463 100644 --- a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/AbstractCoordinatorTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/AbstractCoordinatorTest.java @@ -45,11 +45,16 @@ import java.util.List; import java.util.Map; import java.util.concurrent.CountDownLatch; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.Executors; +import java.util.concurrent.Future; import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicBoolean; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertNotSame; +import static org.junit.Assert.assertSame; import static org.junit.Assert.assertTrue; import static org.junit.Assert.fail; @@ -59,9 +64,9 @@ public class AbstractCoordinatorTest { private static final int REBALANCE_TIMEOUT_MS = 60000; private static final int SESSION_TIMEOUT_MS = 10000; private static final int HEARTBEAT_INTERVAL_MS = 3000; - private static final long RETRY_BACKOFF_MS = 20; - private static final long LONG_RETRY_BACKOFF_MS = 10000; - private static final long REQUEST_TIMEOUT_MS = 40000; + private static final int RETRY_BACKOFF_MS = 100; + private static final int LONG_RETRY_BACKOFF_MS = 10000; + private static final int REQUEST_TIMEOUT_MS = 40000; private static final String GROUP_ID = "dummy-group"; private static final String METRIC_GROUP_PREFIX = "consumer"; @@ -72,27 +77,35 @@ public class AbstractCoordinatorTest { private ConsumerNetworkClient consumerClient; private DummyCoordinator coordinator; - private void setupCoordinator(long retryBackoffMs) { + private void setupCoordinator() { + setupCoordinator(RETRY_BACKOFF_MS, REBALANCE_TIMEOUT_MS); + } + + private void setupCoordinator(int retryBackoffMs) { + setupCoordinator(retryBackoffMs, REBALANCE_TIMEOUT_MS); + } + + private void setupCoordinator(int retryBackoffMs, int rebalanceTimeoutMs) { this.mockTime = new MockTime(); this.mockClient = new MockClient(mockTime); - Metadata metadata = new Metadata(100L, 60 * 60 * 1000L, true); + Metadata metadata = new Metadata(retryBackoffMs, 60 * 60 * 1000L, true); this.consumerClient = new ConsumerNetworkClient(new LogContext(), mockClient, metadata, mockTime, retryBackoffMs, REQUEST_TIMEOUT_MS, HEARTBEAT_INTERVAL_MS); Metrics metrics = new Metrics(); Cluster cluster = TestUtils.singletonCluster("topic", 1); - metadata.update(cluster, Collections.emptySet(), mockTime.milliseconds()); + metadata.update(cluster, Collections.emptySet(), mockTime.milliseconds()); this.node = cluster.nodes().get(0); mockClient.setNode(node); this.coordinatorNode = new Node(Integer.MAX_VALUE - node.id(), node.host(), node.port()); - this.coordinator = new DummyCoordinator(consumerClient, metrics, mockTime); + this.coordinator = new DummyCoordinator(consumerClient, metrics, mockTime, rebalanceTimeoutMs, retryBackoffMs); } @Test public void testCoordinatorDiscoveryBackoff() { - setupCoordinator(RETRY_BACKOFF_MS); + setupCoordinator(); mockClient.prepareResponse(groupCoordinatorResponse(node, Errors.NONE)); mockClient.prepareResponse(groupCoordinatorResponse(node, Errors.NONE)); @@ -108,9 +121,69 @@ public void testCoordinatorDiscoveryBackoff() { assertTrue(endTime - initialTime >= RETRY_BACKOFF_MS); } + @Test + public void testTimeoutAndRetryJoinGroupIfNeeded() throws Exception { + setupCoordinator(); + mockClient.prepareResponse(groupCoordinatorResponse(node, Errors.NONE)); + coordinator.ensureCoordinatorReady(0); + + ExecutorService executor = Executors.newFixedThreadPool(1); + try { + long firstAttemptStartMs = mockTime.milliseconds(); + Future firstAttempt = executor.submit(() -> + coordinator.joinGroupIfNeeded(REQUEST_TIMEOUT_MS, firstAttemptStartMs)); + + mockTime.sleep(REQUEST_TIMEOUT_MS); + assertFalse(firstAttempt.get()); + assertTrue(consumerClient.hasPendingRequests(coordinatorNode)); + + mockClient.respond(joinGroupFollowerResponse(1, "memberId", "leaderId", Errors.NONE)); + mockClient.prepareResponse(syncGroupResponse(Errors.NONE)); + + long secondAttemptMs = mockTime.milliseconds(); + Future secondAttempt = executor.submit(() -> + coordinator.joinGroupIfNeeded(REQUEST_TIMEOUT_MS, secondAttemptMs)); + + assertTrue(secondAttempt.get()); + } finally { + executor.shutdownNow(); + executor.awaitTermination(1000, TimeUnit.MILLISECONDS); + } + } + + @Test + public void testJoinGroupRequestTimeout() { + setupCoordinator(RETRY_BACKOFF_MS, REBALANCE_TIMEOUT_MS); + mockClient.prepareResponse(groupCoordinatorResponse(node, Errors.NONE)); + coordinator.ensureCoordinatorReady(0); + + RequestFuture future = coordinator.sendJoinGroupRequest(); + + mockTime.sleep(REQUEST_TIMEOUT_MS + 1); + assertFalse(consumerClient.poll(future, 0)); + + mockTime.sleep(REBALANCE_TIMEOUT_MS - REQUEST_TIMEOUT_MS + 5000); + assertTrue(consumerClient.poll(future, 0)); + } + + @Test + public void testJoinGroupRequestMaxTimeout() { + // Ensure we can handle the maximum allowed rebalance timeout + + setupCoordinator(RETRY_BACKOFF_MS, Integer.MAX_VALUE); + mockClient.prepareResponse(groupCoordinatorResponse(node, Errors.NONE)); + coordinator.ensureCoordinatorReady(0); + + RequestFuture future = coordinator.sendJoinGroupRequest(); + assertFalse(consumerClient.poll(future, 0)); + + mockTime.sleep(Integer.MAX_VALUE + 1L); + assertTrue(consumerClient.poll(future, 0)); + } + @Test public void testUncaughtExceptionInHeartbeatThread() throws Exception { - setupCoordinator(RETRY_BACKOFF_MS); + setupCoordinator(); mockClient.prepareResponse(groupCoordinatorResponse(node, Errors.NONE)); mockClient.prepareResponse(joinGroupFollowerResponse(1, "memberId", "leaderId", Errors.NONE)); @@ -170,8 +243,8 @@ public boolean matches(AbstractRequest body) { } @Test - public void testLookupCoordinator() throws Exception { - setupCoordinator(RETRY_BACKOFF_MS); + public void testLookupCoordinator() { + setupCoordinator(); mockClient.setNode(null); RequestFuture noBrokersAvailableFuture = coordinator.lookupCoordinator(); @@ -180,16 +253,16 @@ public void testLookupCoordinator() throws Exception { mockClient.setNode(node); RequestFuture future = coordinator.lookupCoordinator(); assertFalse("Request not sent", future.isDone()); - assertTrue("New request sent while one is in progress", future == coordinator.lookupCoordinator()); + assertSame("New request sent while one is in progress", future, coordinator.lookupCoordinator()); mockClient.prepareResponse(groupCoordinatorResponse(node, Errors.NONE)); coordinator.ensureCoordinatorReady(Long.MAX_VALUE); - assertTrue("New request not sent after previous completed", future != coordinator.lookupCoordinator()); + assertNotSame("New request not sent after previous completed", future, coordinator.lookupCoordinator()); } @Test public void testWakeupAfterJoinGroupSent() throws Exception { - setupCoordinator(RETRY_BACKOFF_MS); + setupCoordinator(); mockClient.prepareResponse(groupCoordinatorResponse(node, Errors.NONE)); mockClient.prepareResponse(new MockClient.RequestMatcher() { @@ -227,7 +300,7 @@ public boolean matches(AbstractRequest body) { @Test public void testWakeupAfterJoinGroupSentExternalCompletion() throws Exception { - setupCoordinator(RETRY_BACKOFF_MS); + setupCoordinator(); mockClient.prepareResponse(groupCoordinatorResponse(node, Errors.NONE)); mockClient.prepareResponse(new MockClient.RequestMatcher() { @@ -267,7 +340,7 @@ public boolean matches(AbstractRequest body) { @Test public void testWakeupAfterJoinGroupReceived() throws Exception { - setupCoordinator(RETRY_BACKOFF_MS); + setupCoordinator(); mockClient.prepareResponse(groupCoordinatorResponse(node, Errors.NONE)); mockClient.prepareResponse(new MockClient.RequestMatcher() { @@ -303,7 +376,7 @@ public boolean matches(AbstractRequest body) { @Test public void testWakeupAfterJoinGroupReceivedExternalCompletion() throws Exception { - setupCoordinator(RETRY_BACKOFF_MS); + setupCoordinator(); mockClient.prepareResponse(groupCoordinatorResponse(node, Errors.NONE)); mockClient.prepareResponse(new MockClient.RequestMatcher() { @@ -341,7 +414,7 @@ public boolean matches(AbstractRequest body) { @Test public void testWakeupAfterSyncGroupSent() throws Exception { - setupCoordinator(RETRY_BACKOFF_MS); + setupCoordinator(); mockClient.prepareResponse(groupCoordinatorResponse(node, Errors.NONE)); mockClient.prepareResponse(joinGroupFollowerResponse(1, "memberId", "leaderId", Errors.NONE)); @@ -379,7 +452,7 @@ public boolean matches(AbstractRequest body) { @Test public void testWakeupAfterSyncGroupSentExternalCompletion() throws Exception { - setupCoordinator(RETRY_BACKOFF_MS); + setupCoordinator(); mockClient.prepareResponse(groupCoordinatorResponse(node, Errors.NONE)); mockClient.prepareResponse(joinGroupFollowerResponse(1, "memberId", "leaderId", Errors.NONE)); @@ -419,7 +492,7 @@ public boolean matches(AbstractRequest body) { @Test public void testWakeupAfterSyncGroupReceived() throws Exception { - setupCoordinator(RETRY_BACKOFF_MS); + setupCoordinator(); mockClient.prepareResponse(groupCoordinatorResponse(node, Errors.NONE)); mockClient.prepareResponse(joinGroupFollowerResponse(1, "memberId", "leaderId", Errors.NONE)); @@ -455,7 +528,7 @@ public boolean matches(AbstractRequest body) { @Test public void testWakeupAfterSyncGroupReceivedExternalCompletion() throws Exception { - setupCoordinator(RETRY_BACKOFF_MS); + setupCoordinator(); mockClient.prepareResponse(groupCoordinatorResponse(node, Errors.NONE)); mockClient.prepareResponse(joinGroupFollowerResponse(1, "memberId", "leaderId", Errors.NONE)); @@ -491,7 +564,7 @@ public boolean matches(AbstractRequest body) { @Test public void testWakeupInOnJoinComplete() throws Exception { - setupCoordinator(RETRY_BACKOFF_MS); + setupCoordinator(); coordinator.wakeupOnJoinComplete = true; mockClient.prepareResponse(groupCoordinatorResponse(node, Errors.NONE)); @@ -522,7 +595,7 @@ public void testWakeupInOnJoinComplete() throws Exception { @Test public void testAuthenticationErrorInEnsureCoordinatorReady() { - setupCoordinator(RETRY_BACKOFF_MS); + setupCoordinator(); mockClient.createPendingAuthenticationError(node, 300); @@ -583,9 +656,11 @@ public static class DummyCoordinator extends AbstractCoordinator { public DummyCoordinator(ConsumerNetworkClient client, Metrics metrics, - Time time) { - super(new LogContext(), client, GROUP_ID, REBALANCE_TIMEOUT_MS, SESSION_TIMEOUT_MS, - HEARTBEAT_INTERVAL_MS, metrics, METRIC_GROUP_PREFIX, time, RETRY_BACKOFF_MS, false); + Time time, + int rebalanceTimeoutMs, + int retryBackoffMs) { + super(new LogContext(), client, GROUP_ID, rebalanceTimeoutMs, SESSION_TIMEOUT_MS, + HEARTBEAT_INTERVAL_MS, metrics, METRIC_GROUP_PREFIX, time, retryBackoffMs, false); } @Override diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/ConsumerCoordinatorTest.java b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/ConsumerCoordinatorTest.java index 32da34a166181..7f7e0f27d0f2a 100644 --- a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/ConsumerCoordinatorTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/ConsumerCoordinatorTest.java @@ -30,6 +30,8 @@ import org.apache.kafka.clients.consumer.RoundRobinAssignor; 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.Node; import org.apache.kafka.common.TopicPartition; import org.apache.kafka.common.errors.ApiException; @@ -55,6 +57,7 @@ import org.apache.kafka.common.requests.SyncGroupResponse; import org.apache.kafka.common.utils.LogContext; import org.apache.kafka.common.utils.MockTime; +import org.apache.kafka.test.TestCondition; import org.apache.kafka.test.TestUtils; import org.junit.After; import org.junit.Before; @@ -77,6 +80,7 @@ import java.util.concurrent.TimeoutException; import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicInteger; +import java.util.concurrent.atomic.AtomicReference; import java.util.regex.Pattern; import static java.util.Collections.singleton; @@ -90,16 +94,20 @@ public class ConsumerCoordinatorTest { - private String topic1 = "test1"; - private String topic2 = "test2"; - private String groupId = "test-group"; - private TopicPartition t1p = new TopicPartition(topic1, 0); - private TopicPartition t2p = new TopicPartition(topic2, 0); - private int rebalanceTimeoutMs = 60000; - private int sessionTimeoutMs = 10000; - private int heartbeatIntervalMs = 5000; - private long retryBackoffMs = 100; - private int autoCommitIntervalMs = 2000; + private final String topic1 = "test1"; + private final String topic2 = "test2"; + private final TopicPartition t1p = new TopicPartition(topic1, 0); + private final TopicPartition t2p = new TopicPartition(topic2, 0); + private final String groupId = "test-group"; + private final int rebalanceTimeoutMs = 60000; + private final int sessionTimeoutMs = 10000; + private final int heartbeatIntervalMs = 5000; + private final long retryBackoffMs = 100; + private final int autoCommitIntervalMs = 2000; + private final int requestTimeoutMs = 30000; + private final Heartbeat heartbeat = new Heartbeat(sessionTimeoutMs, heartbeatIntervalMs, + rebalanceTimeoutMs, retryBackoffMs); + private MockPartitionAssignor partitionAssignor = new MockPartitionAssignor(); private List assignors = Collections.singletonList(partitionAssignor); private MockTime time; @@ -126,7 +134,8 @@ public void setup() { this.metadata = new Metadata(0, Long.MAX_VALUE, true); this.metadata.update(cluster, Collections.emptySet(), time.milliseconds()); this.client = new MockClient(time, metadata); - this.consumerClient = new ConsumerNetworkClient(new LogContext(), client, metadata, time, 100, 1000, Integer.MAX_VALUE); + this.consumerClient = new ConsumerNetworkClient(new LogContext(), client, metadata, time, 100, + requestTimeoutMs, Integer.MAX_VALUE); this.metrics = new Metrics(time); this.rebalanceListener = new MockRebalanceListener(); this.mockOffsetCommitCallback = new MockCommitCallback(); @@ -139,6 +148,7 @@ public void setup() { @After public void teardown() { this.metrics.close(); + this.coordinator.close(0); } @Test @@ -436,7 +446,7 @@ public boolean matches(AbstractRequest body) { coordinator.poll(Long.MAX_VALUE); assertFalse(coordinator.rejoinNeededOrPending()); - assertEquals(2, subscriptions.assignedPartitions().size()); + assertEquals(2, subscriptions.numAssignedPartitions()); assertEquals(2, subscriptions.groupSubscription().size()); assertEquals(2, subscriptions.subscription().size()); assertEquals(1, rebalanceListener.revokedCount); @@ -507,6 +517,50 @@ public boolean matches(AbstractRequest body) { assertEquals(newAssignmentSet, rebalanceListener.assigned); } + @Test + public void testForceMetadataRefreshForPatternSubscriptionDuringRebalance() { + // Set up a non-leader consumer with pattern subscription and a cluster containing one topic matching the + // pattern. + final String consumerId = "consumer"; + + subscriptions.subscribe(Pattern.compile(".*"), rebalanceListener); + metadata.update(TestUtils.singletonCluster(topic1, 1), Collections.emptySet(), + time.milliseconds()); + assertEquals(singleton(topic1), subscriptions.subscription()); + + client.prepareResponse(groupCoordinatorResponse(node, Errors.NONE)); + coordinator.ensureCoordinatorReady(Long.MAX_VALUE); + + // Instrument the test so that metadata will contain two topics after next refresh. + client.prepareMetadataUpdate(cluster, Collections.emptySet()); + + client.prepareResponse(joinGroupFollowerResponse(1, consumerId, "leader", Errors.NONE)); + client.prepareResponse(new MockClient.RequestMatcher() { + @Override + public boolean matches(AbstractRequest body) { + SyncGroupRequest sync = (SyncGroupRequest) body; + return sync.memberId().equals(consumerId) && + sync.generationId() == 1 && + sync.groupAssignment().isEmpty(); + } + }, syncGroupResponse(singletonList(t1p), Errors.NONE)); + + partitionAssignor.prepare(singletonMap(consumerId, singletonList(t1p))); + + // This will trigger rebalance. + coordinator.poll(Long.MAX_VALUE); + + // Make sure that the metadata was refreshed during the rebalance and thus subscriptions now contain two topics. + final Set updatedSubscriptionSet = new HashSet<>(Arrays.asList(topic1, topic2)); + assertEquals(updatedSubscriptionSet, subscriptions.subscription()); + + // Refresh the metadata again. Since there have been no changes since the last refresh, it won't trigger + // rebalance again. + metadata.requestUpdate(); + client.poll(Long.MAX_VALUE, time.milliseconds()); + assertFalse(coordinator.rejoinNeededOrPending()); + } + @Test public void testWakeupDuringJoin() { final String consumerId = "leader"; @@ -566,7 +620,7 @@ public boolean matches(AbstractRequest body) { } }, syncGroupResponse(singletonList(t1p), Errors.NONE)); - coordinator.joinGroupIfNeeded(Long.MAX_VALUE); + coordinator.joinGroupIfNeeded(Long.MAX_VALUE, time.milliseconds()); assertFalse(coordinator.rejoinNeededOrPending()); assertEquals(singleton(t1p), subscriptions.assignedPartitions()); @@ -577,6 +631,35 @@ public boolean matches(AbstractRequest body) { assertEquals(singleton(t1p), rebalanceListener.assigned); } + @Test + public void testUpdateLastHeartbeatPollWhenCoordinatorUnknown() throws Exception { + // If we are part of an active group and we cannot find the coordinator, we should nevertheless + // continue to update the last poll time so that we do not expire the consumer + subscriptions.subscribe(singleton(topic1), rebalanceListener); + + client.prepareResponse(groupCoordinatorResponse(node, Errors.NONE)); + coordinator.ensureCoordinatorReady(Long.MAX_VALUE); + + // Join the group, but signal a coordinator change after the first heartbeat + client.prepareResponse(joinGroupFollowerResponse(1, "consumer", "leader", Errors.NONE)); + client.prepareResponse(syncGroupResponse(singletonList(t1p), Errors.NONE)); + client.prepareResponse(heartbeatResponse(Errors.NOT_COORDINATOR)); + + coordinator.poll(Long.MAX_VALUE); + time.sleep(heartbeatIntervalMs); + + // Await the first heartbeat which forces us to find a new coordinator + TestUtils.waitForCondition(() -> !client.hasPendingResponses(), + "Failed to observe expected heartbeat from background thread"); + + assertTrue(coordinator.coordinatorUnknown()); + assertFalse(coordinator.poll(0)); + assertEquals(time.milliseconds(), heartbeat.lastPollTime()); + + time.sleep(rebalanceTimeoutMs - 1); + assertFalse(heartbeat.pollTimeoutExpired(time.milliseconds())); + } + @Test public void testPatternJoinGroupFollower() { final String consumerId = "consumer"; @@ -603,12 +686,12 @@ public boolean matches(AbstractRequest body) { } }, syncGroupResponse(Arrays.asList(t1p, t2p), Errors.NONE)); // expect client to force updating the metadata, if yes gives it both topics - client.prepareMetadataUpdate(cluster, Collections.emptySet()); + client.prepareMetadataUpdate(cluster, Collections.emptySet()); - coordinator.joinGroupIfNeeded(Long.MAX_VALUE); + coordinator.joinGroupIfNeeded(Long.MAX_VALUE, time.milliseconds()); assertFalse(coordinator.rejoinNeededOrPending()); - assertEquals(2, subscriptions.assignedPartitions().size()); + assertEquals(2, subscriptions.numAssignedPartitions()); assertEquals(2, subscriptions.subscription().size()); assertEquals(1, rebalanceListener.revokedCount); assertEquals(1, rebalanceListener.assignedCount); @@ -671,8 +754,8 @@ public void testUnexpectedErrorOnSyncGroup() { // join initially, but let coordinator rebalance on sync client.prepareResponse(joinGroupFollowerResponse(1, consumerId, "leader", Errors.NONE)); - client.prepareResponse(syncGroupResponse(Collections.emptyList(), Errors.UNKNOWN_SERVER_ERROR)); - coordinator.joinGroupIfNeeded(Long.MAX_VALUE); + client.prepareResponse(syncGroupResponse(Collections.emptyList(), Errors.UNKNOWN_SERVER_ERROR)); + coordinator.joinGroupIfNeeded(Long.MAX_VALUE, time.milliseconds()); } @Test @@ -698,7 +781,7 @@ public boolean matches(AbstractRequest body) { }, joinGroupFollowerResponse(2, consumerId, "leader", Errors.NONE)); client.prepareResponse(syncGroupResponse(singletonList(t1p), Errors.NONE)); - coordinator.joinGroupIfNeeded(Long.MAX_VALUE); + coordinator.joinGroupIfNeeded(Long.MAX_VALUE, time.milliseconds()); assertFalse(coordinator.rejoinNeededOrPending()); assertEquals(singleton(t1p), subscriptions.assignedPartitions()); @@ -721,7 +804,7 @@ public void testRebalanceInProgressOnSyncGroup() { client.prepareResponse(joinGroupFollowerResponse(2, consumerId, "leader", Errors.NONE)); client.prepareResponse(syncGroupResponse(singletonList(t1p), Errors.NONE)); - coordinator.joinGroupIfNeeded(Long.MAX_VALUE); + coordinator.joinGroupIfNeeded(Long.MAX_VALUE, time.milliseconds()); assertFalse(coordinator.rejoinNeededOrPending()); assertEquals(singleton(t1p), subscriptions.assignedPartitions()); @@ -750,7 +833,7 @@ public boolean matches(AbstractRequest body) { }, joinGroupFollowerResponse(2, consumerId, "leader", Errors.NONE)); client.prepareResponse(syncGroupResponse(singletonList(t1p), Errors.NONE)); - coordinator.joinGroupIfNeeded(Long.MAX_VALUE); + coordinator.joinGroupIfNeeded(Long.MAX_VALUE, time.milliseconds()); assertFalse(coordinator.rejoinNeededOrPending()); assertEquals(singleton(t1p), subscriptions.assignedPartitions()); @@ -839,6 +922,57 @@ public boolean matches(AbstractRequest body) { assertEquals(new HashSet<>(Arrays.asList(tp1, tp2)), subscriptions.assignedPartitions()); } + @Test + public void testWakeupFromAssignmentCallback() { + ConsumerCoordinator coordinator = buildCoordinator(new Metrics(), assignors, + ConsumerConfig.DEFAULT_EXCLUDE_INTERNAL_TOPICS, false, true); + + final String topic = "topic1"; + TopicPartition partition = new TopicPartition(topic, 0); + final String consumerId = "follower"; + Set topics = Collections.singleton(topic); + MockRebalanceListener rebalanceListener = new MockRebalanceListener() { + @Override + public void onPartitionsAssigned(Collection partitions) { + boolean raiseWakeup = this.assignedCount == 0; + super.onPartitionsAssigned(partitions); + + if (raiseWakeup) + throw new WakeupException(); + } + }; + + subscriptions.subscribe(topics, rebalanceListener); + metadata.setTopics(topics); + + // we only have metadata for one topic initially + metadata.update(TestUtils.singletonCluster(topic, 1), Collections.emptySet(), time.milliseconds()); + + client.prepareResponse(groupCoordinatorResponse(node, Errors.NONE)); + coordinator.ensureCoordinatorReady(Long.MAX_VALUE); + + // prepare initial rebalance + partitionAssignor.prepare(singletonMap(consumerId, Collections.singletonList(partition))); + + client.prepareResponse(joinGroupFollowerResponse(1, consumerId, "leader", Errors.NONE)); + client.prepareResponse(syncGroupResponse(Collections.singletonList(partition), Errors.NONE)); + + + // The first call to poll should raise the exception from the rebalance listener + try { + coordinator.poll(Long.MAX_VALUE); + fail("Expected exception thrown from assignment callback"); + } catch (WakeupException e) { + } + + // The second call should retry the assignment callback and succeed + coordinator.poll(Long.MAX_VALUE); + + assertFalse(coordinator.rejoinNeededOrPending()); + assertEquals(1, rebalanceListener.revokedCount); + assertEquals(2, rebalanceListener.assignedCount); + } + @Test public void testRebalanceAfterTopicUnavailableWithSubscribe() { unavailableTopicTest(false, false, Collections.emptySet()); @@ -937,7 +1071,7 @@ public void testRejoinGroup() { subscriptions.subscribe(new HashSet<>(Arrays.asList(topic1, otherTopic)), rebalanceListener); client.prepareResponse(joinGroupFollowerResponse(2, "consumer", "leader", Errors.NONE)); client.prepareResponse(syncGroupResponse(singletonList(t1p), Errors.NONE)); - coordinator.joinGroupIfNeeded(Long.MAX_VALUE); + coordinator.joinGroupIfNeeded(Long.MAX_VALUE, time.milliseconds()); assertEquals(2, rebalanceListener.revokedCount); assertEquals(singleton(t1p), rebalanceListener.revoked); @@ -957,7 +1091,7 @@ public void testDisconnectInJoin() { client.prepareResponse(groupCoordinatorResponse(node, Errors.NONE)); client.prepareResponse(joinGroupFollowerResponse(1, "consumer", "leader", Errors.NONE)); client.prepareResponse(syncGroupResponse(singletonList(t1p), Errors.NONE)); - coordinator.joinGroupIfNeeded(Long.MAX_VALUE); + coordinator.joinGroupIfNeeded(Long.MAX_VALUE, time.milliseconds()); assertFalse(coordinator.rejoinNeededOrPending()); assertEquals(singleton(t1p), subscriptions.assignedPartitions()); @@ -975,7 +1109,7 @@ public void testInvalidSessionTimeout() { // coordinator doesn't like the session timeout client.prepareResponse(joinGroupFollowerResponse(0, "consumer", "", Errors.INVALID_SESSION_TIMEOUT)); - coordinator.joinGroupIfNeeded(Long.MAX_VALUE); + coordinator.joinGroupIfNeeded(Long.MAX_VALUE, time.milliseconds()); } @Test @@ -1132,7 +1266,7 @@ public void testAutoCommitDynamicAssignmentRebalance() { client.prepareResponse(joinGroupFollowerResponse(1, consumerId, "leader", Errors.NONE)); client.prepareResponse(syncGroupResponse(singletonList(t1p), Errors.NONE)); - coordinator.joinGroupIfNeeded(Long.MAX_VALUE); + coordinator.joinGroupIfNeeded(Long.MAX_VALUE, time.milliseconds()); subscriptions.seek(t1p, 100); @@ -1607,6 +1741,60 @@ public void testProtocolMetadataOrder() { } } + @Test + public void testThreadSafeAssignedPartitionsMetric() throws Exception { + // Get the assigned-partitions metric + final Metric metric = metrics.metric(new MetricName("assigned-partitions", "consumer" + groupId + "-coordinator-metrics", + "", Collections.emptyMap())); + + // Start polling the metric in the background + final AtomicBoolean doStop = new AtomicBoolean(); + final AtomicReference exceptionHolder = new AtomicReference<>(); + final AtomicInteger observedSize = new AtomicInteger(); + + Thread poller = new Thread() { + @Override + public void run() { + // Poll as fast as possible to reproduce ConcurrentModificationException + while (!doStop.get()) { + try { + int size = ((Double) metric.metricValue()).intValue(); + observedSize.set(size); + } catch (Exception e) { + exceptionHolder.set(e); + return; + } + } + } + }; + poller.start(); + + // Assign two partitions to trigger a metric change that can lead to ConcurrentModificationException + client.prepareResponse(groupCoordinatorResponse(node, Errors.NONE)); + coordinator.ensureCoordinatorReady(Long.MAX_VALUE); + + // Change the assignment several times to increase likelihood of concurrent updates + Set partitions = new HashSet<>(); + int totalPartitions = 10; + for (int partition = 0; partition < totalPartitions; partition++) { + partitions.add(new TopicPartition(topic1, partition)); + subscriptions.assignFromUser(partitions); + } + + // Wait for the metric poller to observe the final assignment change or raise an error + TestUtils.waitForCondition(new TestCondition() { + @Override + public boolean conditionMet() { + return observedSize.get() == totalPartitions || exceptionHolder.get() != null; + } + }, "Failed to observe expected assignment change"); + + doStop.set(true); + poller.join(); + + assertNull("Failed fetching the metric at least once", exceptionHolder.get()); + } + @Test public void testCloseDynamicAssignment() throws Exception { ConsumerCoordinator coordinator = prepareCoordinatorForCloseTest(true, true, true); @@ -1630,14 +1818,14 @@ public void testCloseCoordinatorNotKnownManualAssignment() throws Exception { ConsumerCoordinator coordinator = prepareCoordinatorForCloseTest(false, true, true); makeCoordinatorUnknown(coordinator, Errors.NOT_COORDINATOR); time.sleep(autoCommitIntervalMs); - closeVerifyTimeout(coordinator, 1000, 60000, 1000, 1000); + closeVerifyTimeout(coordinator, 1000, 1000, 1000); } @Test public void testCloseCoordinatorNotKnownNoCommits() throws Exception { ConsumerCoordinator coordinator = prepareCoordinatorForCloseTest(true, false, true); makeCoordinatorUnknown(coordinator, Errors.NOT_COORDINATOR); - closeVerifyTimeout(coordinator, 1000, 60000, 0, 0); + closeVerifyTimeout(coordinator, 1000, 0, 0); } @Test @@ -1645,14 +1833,14 @@ public void testCloseCoordinatorNotKnownWithCommits() throws Exception { ConsumerCoordinator coordinator = prepareCoordinatorForCloseTest(true, true, true); makeCoordinatorUnknown(coordinator, Errors.NOT_COORDINATOR); time.sleep(autoCommitIntervalMs); - closeVerifyTimeout(coordinator, 1000, 60000, 1000, 1000); + closeVerifyTimeout(coordinator, 1000, 1000, 1000); } @Test public void testCloseCoordinatorUnavailableNoCommits() throws Exception { ConsumerCoordinator coordinator = prepareCoordinatorForCloseTest(true, false, true); makeCoordinatorUnknown(coordinator, Errors.COORDINATOR_NOT_AVAILABLE); - closeVerifyTimeout(coordinator, 1000, 60000, 0, 0); + closeVerifyTimeout(coordinator, 1000, 0, 0); } @Test @@ -1660,7 +1848,7 @@ public void testCloseTimeoutCoordinatorUnavailableForCommit() throws Exception { ConsumerCoordinator coordinator = prepareCoordinatorForCloseTest(true, true, true); makeCoordinatorUnknown(coordinator, Errors.COORDINATOR_NOT_AVAILABLE); time.sleep(autoCommitIntervalMs); - closeVerifyTimeout(coordinator, 1000, 60000, 1000, 1000); + closeVerifyTimeout(coordinator, 1000, 1000, 1000); } @Test @@ -1668,37 +1856,36 @@ public void testCloseMaxWaitCoordinatorUnavailableForCommit() throws Exception { ConsumerCoordinator coordinator = prepareCoordinatorForCloseTest(true, true, true); makeCoordinatorUnknown(coordinator, Errors.COORDINATOR_NOT_AVAILABLE); time.sleep(autoCommitIntervalMs); - closeVerifyTimeout(coordinator, Long.MAX_VALUE, 60000, 60000, 60000); + closeVerifyTimeout(coordinator, Long.MAX_VALUE, requestTimeoutMs, requestTimeoutMs); } @Test public void testCloseNoResponseForCommit() throws Exception { ConsumerCoordinator coordinator = prepareCoordinatorForCloseTest(true, true, true); time.sleep(autoCommitIntervalMs); - closeVerifyTimeout(coordinator, Long.MAX_VALUE, 60000, 60000, 60000); + closeVerifyTimeout(coordinator, Long.MAX_VALUE, requestTimeoutMs, requestTimeoutMs); } @Test public void testCloseNoResponseForLeaveGroup() throws Exception { ConsumerCoordinator coordinator = prepareCoordinatorForCloseTest(true, false, true); - closeVerifyTimeout(coordinator, Long.MAX_VALUE, 60000, 60000, 60000); + closeVerifyTimeout(coordinator, Long.MAX_VALUE, requestTimeoutMs, requestTimeoutMs); } @Test public void testCloseNoWait() throws Exception { ConsumerCoordinator coordinator = prepareCoordinatorForCloseTest(true, true, true); time.sleep(autoCommitIntervalMs); - closeVerifyTimeout(coordinator, 0, 60000, 0, 0); + closeVerifyTimeout(coordinator, 0, 0, 0); } @Test public void testHeartbeatThreadClose() throws Exception { - groupId = "testCloseTimeoutWithHeartbeatThread"; ConsumerCoordinator coordinator = prepareCoordinatorForCloseTest(true, true, true); coordinator.ensureActiveGroup(); time.sleep(heartbeatIntervalMs + 100); Thread.yield(); // Give heartbeat thread a chance to attempt heartbeat - closeVerifyTimeout(coordinator, Long.MAX_VALUE, 60000, 60000, 60000); + closeVerifyTimeout(coordinator, Long.MAX_VALUE, requestTimeoutMs, requestTimeoutMs); Thread[] threads = new Thread[Thread.activeCount()]; int threadCount = Thread.enumerate(threads); for (int i = 0; i < threadCount; i++) @@ -1736,7 +1923,7 @@ private ConsumerCoordinator prepareCoordinatorForCloseTest(final boolean useGrou subscriptions.subscribe(singleton(topic1), rebalanceListener); client.prepareResponse(joinGroupFollowerResponse(1, consumerId, "leader", Errors.NONE)); client.prepareResponse(syncGroupResponse(singletonList(t1p), Errors.NONE)); - coordinator.joinGroupIfNeeded(Long.MAX_VALUE); + coordinator.joinGroupIfNeeded(Long.MAX_VALUE, time.milliseconds()); } else subscriptions.assignFromUser(singleton(t1p)); @@ -1754,9 +1941,11 @@ private void makeCoordinatorUnknown(ConsumerCoordinator coordinator, Errors erro consumerClient.poll(0); assertTrue(coordinator.coordinatorUnknown()); } + private void closeVerifyTimeout(final ConsumerCoordinator coordinator, - final long closeTimeoutMs, final long requestTimeoutMs, - long expectedMinTimeMs, long expectedMaxTimeMs) throws Exception { + final long closeTimeoutMs, + final long expectedMinTimeMs, + final long expectedMaxTimeMs) throws Exception { ExecutorService executor = Executors.newSingleThreadExecutor(); try { boolean coordinatorUnknown = coordinator.coordinatorUnknown(); @@ -1827,7 +2016,7 @@ private ConsumerCoordinator buildCoordinator(final Metrics metrics, groupId, rebalanceTimeoutMs, sessionTimeoutMs, - heartbeatIntervalMs, + heartbeat, assignors, metadata, subscriptions, @@ -1865,7 +2054,7 @@ private JoinGroupResponse joinGroupLeaderResponse(int generationId, private JoinGroupResponse joinGroupFollowerResponse(int generationId, String memberId, String leaderId, Errors error) { return new JoinGroupResponse(error, generationId, partitionAssignor.name(), memberId, leaderId, - Collections.emptyMap()); + Collections.emptyMap()); } private SyncGroupResponse syncGroupResponse(List partitions, Errors error) { @@ -1878,7 +2067,7 @@ private OffsetCommitResponse offsetCommitResponse(Map re } private OffsetFetchResponse offsetFetchResponse(Errors topLevelError) { - return new OffsetFetchResponse(topLevelError, Collections.emptyMap()); + return new OffsetFetchResponse(topLevelError, Collections.emptyMap()); } private OffsetFetchResponse offsetFetchResponse(TopicPartition tp, Errors partitionLevelError, String metadata, long offset) { @@ -1903,7 +2092,7 @@ private void joinAsFollowerAndReceiveAssignment(String consumerId, coordinator.ensureCoordinatorReady(Long.MAX_VALUE); client.prepareResponse(joinGroupFollowerResponse(1, consumerId, "leader", Errors.NONE)); client.prepareResponse(syncGroupResponse(assignment, Errors.NONE)); - coordinator.joinGroupIfNeeded(Long.MAX_VALUE); + coordinator.joinGroupIfNeeded(Long.MAX_VALUE, time.milliseconds()); } private void prepareOffsetCommitRequest(Map expectedOffsets, Errors error) { 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 index d0888fa56554b..d5ec38272444a 100644 --- 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 @@ -24,6 +24,7 @@ import org.apache.kafka.common.Node; import org.apache.kafka.common.errors.AuthenticationException; import org.apache.kafka.common.errors.DisconnectException; +import org.apache.kafka.common.errors.TimeoutException; import org.apache.kafka.common.errors.WakeupException; import org.apache.kafka.common.protocol.Errors; import org.apache.kafka.common.requests.HeartbeatRequest; @@ -125,6 +126,26 @@ public void testDisconnectWithInFlightRequests() { assertTrue(future.exception() instanceof DisconnectException); } + @Test + public void testTimeoutUnsentRequest() { + // Delay connection to the node so that the request remains unsent + client.delayReady(node, 1000); + + RequestFuture future = consumerClient.send(node, heartbeat(), 500); + consumerClient.pollNoWakeup(); + + // Ensure the request is pending, but hasn't been sent + assertTrue(consumerClient.hasPendingRequests()); + assertFalse(client.hasInFlightRequests()); + + time.sleep(501); + consumerClient.pollNoWakeup(); + + assertFalse(consumerClient.hasPendingRequests()); + assertTrue(future.failed()); + assertTrue(future.exception() instanceof TimeoutException); + } + @Test public void doNotBlockIfPollConditionIsSatisfied() { NetworkClient mockNetworkClient = EasyMock.mock(NetworkClient.class); @@ -175,7 +196,7 @@ public void blockOnlyForRetryBackoffIfNoInflightRequests() { NetworkClient mockNetworkClient = EasyMock.mock(NetworkClient.class); ConsumerNetworkClient consumerClient = new ConsumerNetworkClient(new LogContext(), - mockNetworkClient, metadata, time, retryBackoffMs, 1000L, Integer.MAX_VALUE); + mockNetworkClient, metadata, time, retryBackoffMs, 1000, Integer.MAX_VALUE); EasyMock.expect(mockNetworkClient.inFlightRequestCount()).andReturn(0); EasyMock.expect(mockNetworkClient.poll(EasyMock.eq(retryBackoffMs), EasyMock.anyLong())).andReturn(Collections.emptyList()); @@ -273,8 +294,8 @@ public void testAwaitForMetadataUpdateWithTimeout() { } @Test - public void sendExpiry() throws InterruptedException { - long unsentExpiryMs = 10; + public void sendExpiry() { + int requestTimeoutMs = 10; final AtomicBoolean isReady = new AtomicBoolean(); final AtomicBoolean disconnected = new AtomicBoolean(); client = new MockClient(time) { @@ -291,13 +312,13 @@ public boolean connectionFailed(Node node) { } }; // Queue first send, sleep long enough for this to expire and then queue second send - consumerClient = new ConsumerNetworkClient(new LogContext(), client, metadata, time, 100, unsentExpiryMs, Integer.MAX_VALUE); + consumerClient = new ConsumerNetworkClient(new LogContext(), client, metadata, time, 100, requestTimeoutMs, Integer.MAX_VALUE); RequestFuture future1 = consumerClient.send(node, heartbeat()); assertEquals(1, consumerClient.pendingRequestCount()); assertEquals(1, consumerClient.pendingRequestCount(node)); assertFalse(future1.isDone()); - time.sleep(unsentExpiryMs + 1); + time.sleep(requestTimeoutMs + 1); RequestFuture future2 = consumerClient.send(node, heartbeat()); assertEquals(2, consumerClient.pendingRequestCount()); assertEquals(2, consumerClient.pendingRequestCount(node)); 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 index 9164daa426c54..f62ee02dce672 100644 --- 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 @@ -19,6 +19,7 @@ import org.apache.kafka.clients.ApiVersions; import org.apache.kafka.clients.ClientRequest; import org.apache.kafka.clients.ClientUtils; +import org.apache.kafka.clients.FetchSessionHandler; import org.apache.kafka.clients.Metadata; import org.apache.kafka.clients.MockClient; import org.apache.kafka.clients.NetworkClient; @@ -84,10 +85,12 @@ import org.apache.kafka.test.MockSelector; import org.apache.kafka.test.TestUtils; import org.junit.After; +import org.junit.Assert; import org.junit.Before; import org.junit.Test; import java.io.DataOutputStream; +import java.lang.reflect.Field; import java.nio.ByteBuffer; import java.nio.charset.StandardCharsets; import java.util.ArrayList; @@ -100,6 +103,13 @@ import java.util.List; import java.util.Map; import java.util.Set; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.Executors; +import java.util.concurrent.Future; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicInteger; +import java.util.function.Function; +import java.util.stream.Collectors; import static java.util.Collections.singleton; import static org.apache.kafka.common.requests.FetchMetadata.INVALID_SESSION_ID; @@ -107,9 +117,11 @@ import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertFalse; import static org.junit.Assert.assertNull; +import static org.junit.Assert.assertNotNull; import static org.junit.Assert.assertTrue; import static org.junit.Assert.fail; + @SuppressWarnings("deprecation") public class FetcherTest { private ConsumerRebalanceListener listener = new NoOpConsumerRebalanceListener(); @@ -147,38 +159,30 @@ public class FetcherTest { private Fetcher fetcher = createFetcher(subscriptions, metrics); private Metrics fetcherMetrics = new Metrics(time); private Fetcher fetcherNoAutoReset = createFetcher(subscriptionsNoAutoReset, fetcherMetrics); + private ExecutorService executorService; @Before public void setup() throws Exception { metadata.update(cluster, Collections.emptySet(), time.milliseconds()); client.setNode(node); - MemoryRecordsBuilder builder = MemoryRecords.builder(ByteBuffer.allocate(1024), CompressionType.NONE, TimestampType.CREATE_TIME, 1L); - builder.append(0L, "key".getBytes(), "value-1".getBytes()); - builder.append(0L, "key".getBytes(), "value-2".getBytes()); - builder.append(0L, "key".getBytes(), "value-3".getBytes()); - records = builder.build(); - - builder = MemoryRecords.builder(ByteBuffer.allocate(1024), CompressionType.NONE, TimestampType.CREATE_TIME, 4L); - builder.append(0L, "key".getBytes(), "value-4".getBytes()); - builder.append(0L, "key".getBytes(), "value-5".getBytes()); - nextRecords = builder.build(); - - builder = MemoryRecords.builder(ByteBuffer.allocate(1024), CompressionType.NONE, TimestampType.CREATE_TIME, 0L); - emptyRecords = builder.build(); - - builder = MemoryRecords.builder(ByteBuffer.allocate(1024), CompressionType.NONE, TimestampType.CREATE_TIME, 4L); - builder.append(0L, "key".getBytes(), "value-0".getBytes()); - partialRecords = builder.build(); + records = buildRecords(1L, 3, 1); + nextRecords = buildRecords(4L, 2, 4); + emptyRecords = buildRecords(0L, 0, 0); + partialRecords = buildRecords(4L, 1, 0); partialRecords.buffer().putInt(Records.SIZE_OFFSET, 10000); } @After - public void teardown() { + public void teardown() throws Exception { this.metrics.close(); this.fetcherMetrics.close(); this.fetcher.close(); this.fetcherMetrics.close(); + if (executorService != null) { + executorService.shutdownNow(); + assertTrue(executorService.awaitTermination(5, TimeUnit.SECONDS)); + } } @Test @@ -1388,11 +1392,56 @@ public void testGetTopicMetadataLeaderNotAvailable() { assertTrue(topicMetadata.containsKey(topicName)); } + @Test + public void testGetTopicMetadataOfflinePartitions() { + MetadataResponse originalResponse = newMetadataResponse(topicName, Errors.NONE); //baseline ok response + + //create a response based on the above one with all partitions being leaderless + List altTopics = new ArrayList<>(); + for (MetadataResponse.TopicMetadata item : originalResponse.topicMetadata()) { + List partitions = item.partitionMetadata(); + List altPartitions = new ArrayList<>(); + for (MetadataResponse.PartitionMetadata p : partitions) { + altPartitions.add(new MetadataResponse.PartitionMetadata( + p.error(), + p.partition(), + null, //no leader + p.replicas(), + p.isr(), + p.offlineReplicas()) + ); + } + MetadataResponse.TopicMetadata alteredTopic = new MetadataResponse.TopicMetadata( + item.error(), + item.topic(), + item.isInternal(), + altPartitions + ); + altTopics.add(alteredTopic); + } + Node controller = originalResponse.controller(); + MetadataResponse altered = new MetadataResponse( + (List) originalResponse.brokers(), + originalResponse.clusterId(), + controller != null ? controller.id() : MetadataResponse.NO_CONTROLLER_ID, + altTopics); + + client.prepareResponse(altered); + + Map> topicMetadata = + fetcher.getTopicMetadata(new MetadataRequest.Builder(Collections.singletonList(topicName), false), 5000L); + + Assert.assertNotNull(topicMetadata); + Assert.assertNotNull(topicMetadata.get(topicName)); + //noinspection ConstantConditions + Assert.assertEquals((int) cluster.partitionCountForTopic(topicName), topicMetadata.get(topicName).size()); + } + /* * Send multiple requests. Verify that the client side quota metrics have the right values */ @Test - public void testQuotaMetrics() throws Exception { + public void testQuotaMetrics() { MockSelector selector = new MockSelector(time); Sensor throttleTimeSensor = Fetcher.throttleTimeSensor(metrics, metricsRegistry); Cluster cluster = TestUtils.singletonCluster("test", 1); @@ -1413,8 +1462,8 @@ public void testQuotaMetrics() throws Exception { for (int i = 1; i <= 3; i++) { int throttleTimeMs = 100 * i; - FetchRequest.Builder builder = FetchRequest.Builder.forConsumer(100, 100, new LinkedHashMap()); - ClientRequest request = client.newClientRequest(node.idString(), builder, time.milliseconds(), true, null); + FetchRequest.Builder builder = FetchRequest.Builder.forConsumer(100, 100, new LinkedHashMap<>()); + ClientRequest request = client.newClientRequest(node.idString(), builder, time.milliseconds(), true); client.send(request, time.milliseconds()); client.poll(1, time.milliseconds()); FetchResponse response = fullFetchResponse(tp0, nextRecords, Errors.NONE, i, throttleTimeMs); @@ -1529,7 +1578,6 @@ public void testReadCommittedLagMetric() { tags.put("topic", tp0.topic()); tags.put("partition", String.valueOf(tp0.partition())); MetricName partitionLagMetric = metrics.metricName("records-lag", metricGroup, tags); - MetricName partitionLagMetricDeprecated = metrics.metricName(tp0 + ".records-lag", metricGroup); Map allMetrics = metrics.metrics(); KafkaMetric recordsFetchLagMax = allMetrics.get(maxLagMetric); @@ -1544,9 +1592,6 @@ public void testReadCommittedLagMetric() { KafkaMetric partitionLag = allMetrics.get(partitionLagMetric); assertEquals(50, partitionLag.value(), EPSILON); - KafkaMetric partitionLagDeprecated = allMetrics.get(partitionLagMetricDeprecated); - assertEquals(50, partitionLagDeprecated.value(), EPSILON); - // recordsFetchLagMax should be lso - offset of the last message after receiving a non-empty FetchResponse MemoryRecordsBuilder builder = MemoryRecords.builder(ByteBuffer.allocate(1024), CompressionType.NONE, TimestampType.CREATE_TIME, 0L); @@ -1559,7 +1604,6 @@ public void testReadCommittedLagMetric() { // verify de-registration of partition lag subscriptions.unsubscribe(); assertFalse(allMetrics.containsKey(partitionLagMetric)); - assertFalse(allMetrics.containsKey(partitionLagMetricDeprecated)); } @Test @@ -1797,6 +1841,48 @@ public void testGetOffsetsForTimes() { testGetOffsetsForTimesWithError(Errors.BROKER_NOT_AVAILABLE, Errors.NONE, 10L, 100L, 10L, 100L); } + @Test + public void testGetOffsetsForTimesWhenSomeTopicPartitionLeadersNotKnownInitially() { + final String anotherTopic = "another-topic"; + final TopicPartition t2p0 = new TopicPartition(anotherTopic, 0); + + client.reset(); + + // Metadata initially has one topic + Cluster cluster = TestUtils.clusterWith(3, topicName, 2); + metadata.update(cluster, Collections.emptySet(), time.milliseconds()); + + // The first metadata refresh should contain one topic + client.prepareMetadataUpdate(cluster, Collections.emptySet(), false); + client.prepareResponseFrom(listOffsetResponse(tp0, Errors.NONE, 1000L, 11L), cluster.leaderFor(tp0)); + client.prepareResponseFrom(listOffsetResponse(tp1, Errors.NONE, 1000L, 32L), cluster.leaderFor(tp1)); + + // Second metadata refresh should contain two topics + Map partitionNumByTopic = new HashMap<>(); + partitionNumByTopic.put(topicName, 2); + partitionNumByTopic.put(anotherTopic, 1); + Cluster updatedCluster = TestUtils.clusterWith(3, partitionNumByTopic); + client.prepareMetadataUpdate(updatedCluster, Collections.emptySet(), false); + client.prepareResponseFrom(listOffsetResponse(t2p0, Errors.NONE, 1000L, 54L), cluster.leaderFor(t2p0)); + + Map timestampToSearch = new HashMap<>(); + timestampToSearch.put(tp0, ListOffsetRequest.LATEST_TIMESTAMP); + timestampToSearch.put(tp1, ListOffsetRequest.LATEST_TIMESTAMP); + timestampToSearch.put(t2p0, ListOffsetRequest.LATEST_TIMESTAMP); + Map offsetAndTimestampMap = + fetcher.offsetsByTimes(timestampToSearch, Long.MAX_VALUE); + + assertNotNull("Expect Fetcher.offsetsByTimes() to return non-null result for " + tp0, + offsetAndTimestampMap.get(tp0)); + assertNotNull("Expect Fetcher.offsetsByTimes() to return non-null result for " + tp1, + offsetAndTimestampMap.get(tp1)); + assertNotNull("Expect Fetcher.offsetsByTimes() to return non-null result for " + t2p0, + offsetAndTimestampMap.get(t2p0)); + assertEquals(11L, offsetAndTimestampMap.get(tp0).offset()); + assertEquals(32L, offsetAndTimestampMap.get(tp1).offset()); + assertEquals(54L, offsetAndTimestampMap.get(t2p0).offset()); + } + @Test(expected = TimeoutException.class) public void testBatchedListOffsetsMetadataErrors() { Map partitionData = new HashMap<>(); @@ -2071,8 +2157,8 @@ protected boolean shouldRetainRecord(RecordBatch recordBatch, Record record) { return record.key() != null; } }, ByteBuffer.allocate(1024), Integer.MAX_VALUE, BufferSupplier.NO_CACHING); - result.output.flip(); - MemoryRecords compactedRecords = MemoryRecords.readableRecords(result.output); + result.outputBuffer().flip(); + MemoryRecords compactedRecords = MemoryRecords.readableRecords(result.outputBuffer()); subscriptions.assignFromUser(singleton(tp0)); subscriptions.seek(tp0, 0); @@ -2324,6 +2410,187 @@ public void testConsumingViaIncrementalFetchRequests() { assertEquals(5, records.get(1).offset()); } + @Test + public void testFetcherConcurrency() throws Exception { + int numPartitions = 20; + Set topicPartitions = new HashSet<>(); + for (int i = 0; i < numPartitions; i++) + topicPartitions.add(new TopicPartition(topicName, i)); + cluster = TestUtils.singletonCluster(topicName, numPartitions); + metadata.update(cluster, Collections.emptySet(), time.milliseconds()); + client.setNode(node); + fetchSize = 10000; + + Fetcher fetcher = new Fetcher( + new LogContext(), + consumerClient, + minBytes, + maxBytes, + maxWaitMs, + fetchSize, + 2 * numPartitions, + true, + new ByteArrayDeserializer(), + new ByteArrayDeserializer(), + metadata, + subscriptions, + metrics, + metricsRegistry, + time, + retryBackoffMs, + requestTimeoutMs, + IsolationLevel.READ_UNCOMMITTED) { + @Override + protected FetchSessionHandler sessionHandler(int id) { + final FetchSessionHandler handler = super.sessionHandler(id); + if (handler == null) + return null; + else { + return new FetchSessionHandler(new LogContext(), id) { + @Override + public Builder newBuilder() { + verifySessionPartitions(); + return handler.newBuilder(); + } + + @Override + public boolean handleResponse(FetchResponse response) { + verifySessionPartitions(); + return handler.handleResponse(response); + } + + @Override + public void handleError(Throwable t) { + verifySessionPartitions(); + handler.handleError(t); + } + + // Verify that session partitions can be traversed safely. + private void verifySessionPartitions() { + try { + Field field = FetchSessionHandler.class.getDeclaredField("sessionPartitions"); + field.setAccessible(true); + LinkedHashMap sessionPartitions = + (LinkedHashMap) field.get(handler); + for (Map.Entry entry : sessionPartitions.entrySet()) { + // If `sessionPartitions` are modified on another thread, Thread.yield will increase the + // possibility of ConcurrentModificationException if appropriate synchronization is not used. + Thread.yield(); + } + } catch (Exception e) { + throw new RuntimeException(e); + } + } + }; + } + } + }; + + subscriptions.assignFromUser(topicPartitions); + topicPartitions.forEach(tp -> subscriptions.seek(tp, 0L)); + + AtomicInteger fetchesRemaining = new AtomicInteger(1000); + executorService = Executors.newSingleThreadExecutor(); + Future future = executorService.submit(() -> { + while (fetchesRemaining.get() > 0) { + synchronized (consumerClient) { + if (!client.requests().isEmpty()) { + ClientRequest request = client.requests().peek(); + FetchRequest fetchRequest = (FetchRequest) request.requestBuilder().build(); + LinkedHashMap> responseMap = new LinkedHashMap<>(); + for (Map.Entry entry : fetchRequest.fetchData().entrySet()) { + TopicPartition tp = entry.getKey(); + long offset = entry.getValue().fetchOffset; + responseMap.put(tp, new FetchResponse.PartitionData<>(Errors.NONE, offset + 2L, offset + 2, + 0L, null, buildRecords(offset, 2, offset))); + } + client.respondToRequest(request, new FetchResponse<>(Errors.NONE, responseMap, 0, 123)); + consumerClient.poll(0); + } + } + } + return fetchesRemaining.get(); + }); + Map nextFetchOffsets = topicPartitions.stream() + .collect(Collectors.toMap(Function.identity(), t -> 0L)); + while (fetchesRemaining.get() > 0 && !future.isDone()) { + if (fetcher.sendFetches() == 1) { + synchronized (consumerClient) { + consumerClient.poll(0); + } + } + if (fetcher.hasCompletedFetches()) { + Map>> fetchedRecords = fetcher.fetchedRecords(); + if (!fetchedRecords.isEmpty()) { + fetchesRemaining.decrementAndGet(); + fetchedRecords.entrySet().forEach(entry -> { + TopicPartition tp = entry.getKey(); + List> records = entry.getValue(); + assertEquals(2, records.size()); + long nextOffset = nextFetchOffsets.get(tp); + assertEquals(nextOffset, records.get(0).offset()); + assertEquals(nextOffset + 1, records.get(1).offset()); + nextFetchOffsets.put(tp, nextOffset + 2); + }); + } + } + } + assertEquals(0, future.get()); + } + + @Test + public void testEmptyControlBatch() { + Fetcher fetcher = createFetcher(subscriptions, new Metrics(), new ByteArrayDeserializer(), + new ByteArrayDeserializer(), Integer.MAX_VALUE, IsolationLevel.READ_COMMITTED); + ByteBuffer buffer = ByteBuffer.allocate(1024); + int currentOffset = 1; + + // Empty control batch should not cause an exception + DefaultRecordBatch.writeEmptyHeader(buffer, RecordBatch.MAGIC_VALUE_V2, 1L, + (short) 0, -1, 0, 0, + RecordBatch.NO_PARTITION_LEADER_EPOCH, TimestampType.CREATE_TIME, time.milliseconds(), + true, true); + + currentOffset += appendTransactionalRecords(buffer, 1L, currentOffset, + new SimpleRecord(time.milliseconds(), "key".getBytes(), "value".getBytes()), + new SimpleRecord(time.milliseconds(), "key".getBytes(), "value".getBytes())); + + commitTransaction(buffer, 1L, currentOffset); + buffer.flip(); + + List abortedTransactions = new ArrayList<>(); + MemoryRecords records = MemoryRecords.readableRecords(buffer); + subscriptions.assignFromUser(singleton(tp0)); + + subscriptions.seek(tp0, 0); + + // normal fetch + assertEquals(1, fetcher.sendFetches()); + assertFalse(fetcher.hasCompletedFetches()); + client.prepareResponse(new MockClient.RequestMatcher() { + @Override + public boolean matches(AbstractRequest body) { + FetchRequest request = (FetchRequest) body; + assertEquals(IsolationLevel.READ_COMMITTED, request.isolationLevel()); + return true; + } + }, fullFetchResponseWithAbortedTransactions(records, abortedTransactions, Errors.NONE, 100L, 100L, 0)); + + consumerClient.poll(0); + assertTrue(fetcher.hasCompletedFetches()); + + Map>> fetchedRecords = fetcher.fetchedRecords(); + assertTrue(fetchedRecords.containsKey(tp0)); + assertEquals(fetchedRecords.get(tp0).size(), 2); + } + + private MemoryRecords buildRecords(long baseOffset, int count, long firstMessageId) { + MemoryRecordsBuilder builder = MemoryRecords.builder(ByteBuffer.allocate(1024), CompressionType.NONE, TimestampType.CREATE_TIME, baseOffset); + for (int i = 0; i < count; i++) + builder.append(0L, "key".getBytes(), ("value-" + (firstMessageId + i)).getBytes()); + return builder.build(); + } + private int appendTransactionalRecords(ByteBuffer buffer, long pid, long baseOffset, int baseSequence, SimpleRecord... records) { MemoryRecordsBuilder builder = MemoryRecords.builder(buffer, RecordBatch.CURRENT_MAGIC_VALUE, CompressionType.NONE, TimestampType.CREATE_TIME, baseOffset, time.milliseconds(), pid, (short) 0, baseSequence, true, 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 index 06cdae7ad6942..7db7820fd6825 100644 --- 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 @@ -26,24 +26,24 @@ public class HeartbeatTest { - private long timeout = 300L; - private long interval = 100L; - private long maxPollInterval = 900L; - private long retryBackoff = 10L; + private int sessionTimeoutMs = 300; + private int heartbeatIntervalMs = 100; + private int maxPollIntervalMs = 900; + private long retryBackoffMs = 10L; private MockTime time = new MockTime(); - private Heartbeat heartbeat = new Heartbeat(timeout, interval, maxPollInterval, retryBackoff); + private Heartbeat heartbeat = new Heartbeat(sessionTimeoutMs, heartbeatIntervalMs, maxPollIntervalMs, retryBackoffMs); @Test public void testShouldHeartbeat() { heartbeat.sentHeartbeat(time.milliseconds()); - time.sleep((long) ((float) interval * 1.1)); + time.sleep((long) ((float) heartbeatIntervalMs * 1.1)); assertTrue(heartbeat.shouldHeartbeat(time.milliseconds())); } @Test public void testShouldNotHeartbeat() { heartbeat.sentHeartbeat(time.milliseconds()); - time.sleep(interval / 2); + time.sleep(heartbeatIntervalMs / 2); assertFalse(heartbeat.shouldHeartbeat(time.milliseconds())); } 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 index 24255e8949f70..05287e0c1ecfd 100644 --- 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 @@ -49,12 +49,14 @@ public class SubscriptionStateTest { public void partitionAssignment() { state.assignFromUser(singleton(tp0)); assertEquals(singleton(tp0), state.assignedPartitions()); + assertEquals(1, state.numAssignedPartitions()); assertFalse(state.hasAllFetchPositions()); state.seek(tp0, 1); assertTrue(state.isFetchable(tp0)); assertEquals(1L, state.position(tp0).longValue()); state.assignFromUser(Collections.emptySet()); assertTrue(state.assignedPartitions().isEmpty()); + assertEquals(0, state.numAssignedPartitions()); assertFalse(state.isAssigned(tp0)); assertFalse(state.isFetchable(tp0)); } @@ -64,28 +66,34 @@ public void partitionAssignmentChangeOnTopicSubscription() { state.assignFromUser(new HashSet<>(Arrays.asList(tp0, tp1))); // assigned partitions should immediately change assertEquals(2, state.assignedPartitions().size()); + assertEquals(2, state.numAssignedPartitions()); assertTrue(state.assignedPartitions().contains(tp0)); assertTrue(state.assignedPartitions().contains(tp1)); state.unsubscribe(); // assigned partitions should immediately change assertTrue(state.assignedPartitions().isEmpty()); + assertEquals(0, state.numAssignedPartitions()); state.subscribe(singleton(topic1), rebalanceListener); // assigned partitions should remain unchanged assertTrue(state.assignedPartitions().isEmpty()); + assertEquals(0, state.numAssignedPartitions()); state.assignFromSubscribed(singleton(t1p0)); // assigned partitions should immediately change assertEquals(singleton(t1p0), state.assignedPartitions()); + assertEquals(1, state.numAssignedPartitions()); state.subscribe(singleton(topic), rebalanceListener); // assigned partitions should remain unchanged assertEquals(singleton(t1p0), state.assignedPartitions()); + assertEquals(1, state.numAssignedPartitions()); state.unsubscribe(); // assigned partitions should immediately change assertTrue(state.assignedPartitions().isEmpty()); + assertEquals(0, state.numAssignedPartitions()); } @Test @@ -93,37 +101,45 @@ public void partitionAssignmentChangeOnPatternSubscription() { state.subscribe(Pattern.compile(".*"), rebalanceListener); // assigned partitions should remain unchanged assertTrue(state.assignedPartitions().isEmpty()); + assertEquals(0, state.numAssignedPartitions()); state.subscribeFromPattern(new HashSet<>(Collections.singletonList(topic))); // assigned partitions should remain unchanged assertTrue(state.assignedPartitions().isEmpty()); + assertEquals(0, state.numAssignedPartitions()); state.assignFromSubscribed(singleton(tp1)); // assigned partitions should immediately change assertEquals(singleton(tp1), state.assignedPartitions()); + assertEquals(1, state.numAssignedPartitions()); assertEquals(singleton(topic), state.subscription()); state.assignFromSubscribed(Collections.singletonList(t1p0)); // assigned partitions should immediately change assertEquals(singleton(t1p0), state.assignedPartitions()); + assertEquals(1, state.numAssignedPartitions()); assertEquals(singleton(topic), state.subscription()); state.subscribe(Pattern.compile(".*t"), rebalanceListener); // assigned partitions should remain unchanged assertEquals(singleton(t1p0), state.assignedPartitions()); + assertEquals(1, state.numAssignedPartitions()); state.subscribeFromPattern(singleton(topic)); // assigned partitions should remain unchanged assertEquals(singleton(t1p0), state.assignedPartitions()); + assertEquals(1, state.numAssignedPartitions()); state.assignFromSubscribed(Collections.singletonList(tp0)); // assigned partitions should immediately change assertEquals(singleton(tp0), state.assignedPartitions()); + assertEquals(1, state.numAssignedPartitions()); assertEquals(singleton(topic), state.subscription()); state.unsubscribe(); // assigned partitions should immediately change assertTrue(state.assignedPartitions().isEmpty()); + assertEquals(0, state.numAssignedPartitions()); } @Test @@ -169,6 +185,7 @@ public void topicSubscription() { state.subscribe(singleton(topic), rebalanceListener); assertEquals(1, state.subscription().size()); assertTrue(state.assignedPartitions().isEmpty()); + assertEquals(0, state.numAssignedPartitions()); assertTrue(state.partitionsAutoAssigned()); state.assignFromSubscribed(singleton(tp0)); state.seek(tp0, 1); @@ -178,6 +195,7 @@ public void topicSubscription() { assertFalse(state.isAssigned(tp0)); assertFalse(state.isFetchable(tp1)); assertEquals(singleton(tp1), state.assignedPartitions()); + assertEquals(1, state.numAssignedPartitions()); } @Test @@ -261,6 +279,7 @@ public void unsubscribeUserSubscribe() { state.unsubscribe(); state.assignFromUser(singleton(tp0)); assertEquals(singleton(tp0), state.assignedPartitions()); + assertEquals(1, state.numAssignedPartitions()); } @Test @@ -269,17 +288,21 @@ public void unsubscription() { state.subscribeFromPattern(new HashSet<>(Arrays.asList(topic, topic1))); state.assignFromSubscribed(singleton(tp1)); assertEquals(singleton(tp1), state.assignedPartitions()); + assertEquals(1, state.numAssignedPartitions()); state.unsubscribe(); assertEquals(0, state.subscription().size()); assertTrue(state.assignedPartitions().isEmpty()); + assertEquals(0, state.numAssignedPartitions()); state.assignFromUser(singleton(tp0)); assertEquals(singleton(tp0), state.assignedPartitions()); + assertEquals(1, state.numAssignedPartitions()); state.unsubscribe(); assertEquals(0, state.subscription().size()); assertTrue(state.assignedPartitions().isEmpty()); + assertEquals(0, state.numAssignedPartitions()); } private static class MockRebalanceListener implements ConsumerRebalanceListener { 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 index 8bfc5e7d28a4c..943a70476b716 100644 --- a/clients/src/test/java/org/apache/kafka/clients/producer/KafkaProducerTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/producer/KafkaProducerTest.java @@ -59,6 +59,7 @@ import java.util.HashMap; import java.util.Map; import java.util.Properties; +import java.util.concurrent.ExecutionException; import java.util.concurrent.ExecutorService; import java.util.concurrent.Executors; import java.util.concurrent.Future; @@ -72,6 +73,27 @@ @RunWith(PowerMockRunner.class) @PowerMockIgnore("javax.management.*") public class KafkaProducerTest { + private String topic = "topic"; + private Collection nodes = Collections.singletonList(new Node(0, "host1", 1000)); + private final Cluster emptyCluster = new Cluster(null, nodes, + Collections.emptySet(), + Collections.emptySet(), + Collections.emptySet()); + private final Cluster onePartitionCluster = new Cluster( + "dummy", + Collections.singletonList(new Node(0, "host1", 1000)), + Collections.singletonList(new PartitionInfo(topic, 0, null, null, null)), + Collections.emptySet(), + Collections.emptySet()); + private final Cluster threePartitionCluster = new Cluster( + "dummy", + Collections.singletonList(new Node(0, "host1", 1000)), + Arrays.asList( + new PartitionInfo(topic, 0, null, null, null), + new PartitionInfo(topic, 1, null, null, null), + new PartitionInfo(topic, 2, null, null, null)), + Collections.emptySet(), + Collections.emptySet()); @Test public void testConstructorWithSerializers() { @@ -269,24 +291,12 @@ public void testMetadataFetch() throws Exception { Metadata metadata = PowerMock.createNiceMock(Metadata.class); MemberModifier.field(KafkaProducer.class, "metadata").set(producer, metadata); - String topic = "topic"; ProducerRecord record = new ProducerRecord<>(topic, "value"); - Collection nodes = Collections.singletonList(new Node(0, "host1", 1000)); - final Cluster emptyCluster = new Cluster(null, nodes, - Collections.emptySet(), - Collections.emptySet(), - Collections.emptySet()); - final Cluster cluster = new Cluster( - "dummy", - Collections.singletonList(new Node(0, "host1", 1000)), - Arrays.asList(new PartitionInfo(topic, 0, null, null, null)), - Collections.emptySet(), - Collections.emptySet()); // Expect exactly one fetch for each attempt to refresh while topic metadata is not available final int refreshAttempts = 5; EasyMock.expect(metadata.fetch()).andReturn(emptyCluster).times(refreshAttempts - 1); - EasyMock.expect(metadata.fetch()).andReturn(cluster).once(); + EasyMock.expect(metadata.fetch()).andReturn(onePartitionCluster).once(); EasyMock.expect(metadata.fetch()).andThrow(new IllegalStateException("Unexpected call to metadata.fetch()")).anyTimes(); PowerMock.replay(metadata); producer.send(record); @@ -294,7 +304,7 @@ public void testMetadataFetch() throws Exception { // Expect exactly one fetch if topic metadata is available PowerMock.reset(metadata); - EasyMock.expect(metadata.fetch()).andReturn(cluster).once(); + EasyMock.expect(metadata.fetch()).andReturn(onePartitionCluster).once(); EasyMock.expect(metadata.fetch()).andThrow(new IllegalStateException("Unexpected call to metadata.fetch()")).anyTimes(); PowerMock.replay(metadata); producer.send(record, null); @@ -302,88 +312,109 @@ public void testMetadataFetch() throws Exception { // Expect exactly one fetch if topic metadata is available PowerMock.reset(metadata); - EasyMock.expect(metadata.fetch()).andReturn(cluster).once(); + EasyMock.expect(metadata.fetch()).andReturn(onePartitionCluster).once(); EasyMock.expect(metadata.fetch()).andThrow(new IllegalStateException("Unexpected call to metadata.fetch()")).anyTimes(); PowerMock.replay(metadata); producer.partitionsFor(topic); PowerMock.verify(metadata); } + @Test + public void testMetadataTimeoutWithMissingTopic() throws Exception { + Properties props = new Properties(); + props.setProperty(ProducerConfig.BOOTSTRAP_SERVERS_CONFIG, "localhost:9999"); + props.setProperty(ProducerConfig.MAX_BLOCK_MS_CONFIG, "600000"); + KafkaProducer producer = new KafkaProducer<>(props, new StringSerializer(), new StringSerializer()); + long refreshBackoffMs = 500L; + long metadataExpireMs = 60000L; + final Metadata metadata = new Metadata(refreshBackoffMs, metadataExpireMs, true, + true, new ClusterResourceListeners()); + final Time time = new MockTime(); + MemberModifier.field(KafkaProducer.class, "metadata").set(producer, metadata); + MemberModifier.field(KafkaProducer.class, "time").set(producer, time); + + Thread t = new Thread() { + @Override + public void run() { + long startTimeMs = System.currentTimeMillis(); + for (int i = 0; i < 10; i++) { + while (!metadata.updateRequested() && System.currentTimeMillis() - startTimeMs < 1000) + yield(); + metadata.update(Cluster.empty(), Collections.singleton(topic), time.milliseconds()); + time.sleep(60 * 1000L); + } + } + }; + t.start(); + // Create a record with a partition higher than the initial (outdated) partition range + ProducerRecord record = new ProducerRecord<>(topic, 2, null, "value"); + try { + producer.send(record).get(); + fail("Expect ExecutionException"); + } catch (ExecutionException e) { + // skip + } + Assert.assertTrue("Topic should still exist in metadata", metadata.containsTopic(topic)); + } + @PrepareOnlyThisForTest(Metadata.class) @Test - public void testMetadataFetchOnStaleMetadata() throws Exception { + public void testMetadataWithPartitionOutOfRange() throws Exception { Properties props = new Properties(); props.setProperty(ProducerConfig.BOOTSTRAP_SERVERS_CONFIG, "localhost:9999"); KafkaProducer producer = new KafkaProducer<>(props, new StringSerializer(), new StringSerializer()); Metadata metadata = PowerMock.createNiceMock(Metadata.class); MemberModifier.field(KafkaProducer.class, "metadata").set(producer, metadata); - String topic = "topic"; - ProducerRecord initialRecord = new ProducerRecord<>(topic, "value"); - // Create a record with a partition higher than the initial (outdated) partition range - ProducerRecord extendedRecord = new ProducerRecord<>(topic, 2, null, "value"); - Collection nodes = Collections.singletonList(new Node(0, "host1", 1000)); - final Cluster emptyCluster = new Cluster(null, nodes, - Collections.emptySet(), - Collections.emptySet(), - Collections.emptySet()); - final Cluster initialCluster = new Cluster( - "dummy", - Collections.singletonList(new Node(0, "host1", 1000)), - Arrays.asList(new PartitionInfo(topic, 0, null, null, null)), - Collections.emptySet(), - Collections.emptySet()); - final Cluster extendedCluster = new Cluster( - "dummy", - Collections.singletonList(new Node(0, "host1", 1000)), - Arrays.asList( - new PartitionInfo(topic, 0, null, null, null), - new PartitionInfo(topic, 1, null, null, null), - new PartitionInfo(topic, 2, null, null, null)), - Collections.emptySet(), - Collections.emptySet()); + ProducerRecord record = new ProducerRecord<>(topic, 2, null, "value"); - // Expect exactly one fetch for each attempt to refresh while topic metadata is not available + // Expect exactly one fetch for each attempt to refresh while topic metadata is stale final int refreshAttempts = 5; - EasyMock.expect(metadata.fetch()).andReturn(emptyCluster).times(refreshAttempts - 1); - EasyMock.expect(metadata.fetch()).andReturn(initialCluster).once(); + EasyMock.expect(metadata.fetch()).andReturn(onePartitionCluster).times(refreshAttempts - 1); + EasyMock.expect(metadata.fetch()).andReturn(threePartitionCluster).once(); EasyMock.expect(metadata.fetch()).andThrow(new IllegalStateException("Unexpected call to metadata.fetch()")).anyTimes(); PowerMock.replay(metadata); - producer.send(initialRecord); + producer.send(record); PowerMock.verify(metadata); + } - // Expect exactly one fetch if topic metadata is available and records are still within range - PowerMock.reset(metadata); - EasyMock.expect(metadata.fetch()).andReturn(initialCluster).once(); - EasyMock.expect(metadata.fetch()).andThrow(new IllegalStateException("Unexpected call to metadata.fetch()")).anyTimes(); - PowerMock.replay(metadata); - producer.send(initialRecord, null); - PowerMock.verify(metadata); + @Test + public void testMetadataTimeoutWithPartitionOutOfRange() throws Exception { + Properties props = new Properties(); + props.setProperty(ProducerConfig.BOOTSTRAP_SERVERS_CONFIG, "localhost:9999"); + props.setProperty(ProducerConfig.MAX_BLOCK_MS_CONFIG, "600000"); + KafkaProducer producer = new KafkaProducer<>(props, new StringSerializer(), new StringSerializer()); - // Expect exactly two fetches if topic metadata is available but metadata response still returns - // the same partition size (either because metadata are still stale at the broker too or because - // there weren't any partitions added in the first place). - PowerMock.reset(metadata); - EasyMock.expect(metadata.fetch()).andReturn(initialCluster).once(); - EasyMock.expect(metadata.fetch()).andReturn(initialCluster).once(); - EasyMock.expect(metadata.fetch()).andThrow(new IllegalStateException("Unexpected call to metadata.fetch()")).anyTimes(); - PowerMock.replay(metadata); + long refreshBackoffMs = 500L; + long metadataExpireMs = 60000L; + final Metadata metadata = new Metadata(refreshBackoffMs, metadataExpireMs, true, + true, new ClusterResourceListeners()); + final Time time = new MockTime(); + MemberModifier.field(KafkaProducer.class, "metadata").set(producer, metadata); + MemberModifier.field(KafkaProducer.class, "time").set(producer, time); + + Thread t = new Thread() { + @Override + public void run() { + long startTimeMs = System.currentTimeMillis(); + for (int i = 0; i < 10; i++) { + while (!metadata.updateRequested() && System.currentTimeMillis() - startTimeMs < 1000) + yield(); + metadata.update(onePartitionCluster, Collections.emptySet(), time.milliseconds()); + time.sleep(60 * 1000L); + } + } + }; + t.start(); + // Create a record with a partition higher than the initial (outdated) partition range + ProducerRecord record = new ProducerRecord<>(topic, 2, null, "value"); try { - producer.send(extendedRecord, null); - fail("Expected KafkaException to be raised"); - } catch (KafkaException e) { - // expected + producer.send(record).get(); + fail("Expect ExecutionException"); + } catch (ExecutionException e) { + // skip } - PowerMock.verify(metadata); - - // Expect exactly two fetches if topic metadata is available but outdated for the given record - PowerMock.reset(metadata); - EasyMock.expect(metadata.fetch()).andReturn(initialCluster).once(); - EasyMock.expect(metadata.fetch()).andReturn(extendedCluster).once(); - EasyMock.expect(metadata.fetch()).andThrow(new IllegalStateException("Unexpected call to metadata.fetch()")).anyTimes(); - PowerMock.replay(metadata); - producer.send(extendedRecord, null); - PowerMock.verify(metadata); + Assert.assertTrue("Topic should still exist in metadata", metadata.containsTopic(topic)); } @Test @@ -609,4 +640,52 @@ public void testOnlyCanExecuteCloseAfterInitTransactionsTimeout() { producer.close(0, TimeUnit.MILLISECONDS); } } + + @Test + public void testCloseWhenWaitingForMetadataUpdate() throws InterruptedException { + Properties props = new Properties(); + props.put(ProducerConfig.MAX_BLOCK_MS_CONFIG, Long.MAX_VALUE); + props.put(ProducerConfig.BOOTSTRAP_SERVERS_CONFIG, "localhost:9000"); + + // Simulate a case where metadata for a particular topic is not available. This will cause KafkaProducer#send to + // block in Metadata#awaitUpdate for the configured max.block.ms. When close() is invoked, KafkaProducer#send should + // return with a KafkaException. + String topicName = "test"; + Time time = new MockTime(); + Cluster cluster = TestUtils.singletonCluster(); + Node node = cluster.nodes().get(0); + Metadata metadata = new Metadata(0, Long.MAX_VALUE, false); + metadata.update(cluster, Collections.emptySet(), time.milliseconds()); + MockClient client = new MockClient(time, metadata); + client.setNode(node); + + Producer producer = new KafkaProducer<>( + new ProducerConfig(ProducerConfig.addSerializerToConfig(props, new StringSerializer(), new StringSerializer())), + new StringSerializer(), new StringSerializer(), metadata, client); + + ExecutorService executor = Executors.newSingleThreadExecutor(); + final AtomicReference sendException = new AtomicReference<>(); + + try { + executor.submit(() -> { + try { + // Metadata for topic "test" will not be available which will cause us to block indefinitely until + // KafkaProducer#close is invoked. + producer.send(new ProducerRecord<>(topicName, "key", "value")); + fail(); + } catch (Exception e) { + sendException.set(e); + } + }); + + // Wait until metadata update for the topic has been requested + TestUtils.waitForCondition(() -> metadata.containsTopic(topicName), "Timeout when waiting for topic to be added to metadata"); + producer.close(0, TimeUnit.MILLISECONDS); + TestUtils.waitForCondition(() -> sendException.get() != null, "No producer exception within timeout"); + assertEquals(KafkaException.class, sendException.get().getClass()); + } finally { + executor.shutdownNow(); + } + } + } 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 27fac280afcbb..eef9d6d2c6649 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 @@ -19,6 +19,7 @@ import org.apache.kafka.clients.consumer.OffsetAndMetadata; import org.apache.kafka.clients.producer.internals.DefaultPartitioner; import org.apache.kafka.common.Cluster; +import org.apache.kafka.common.KafkaException; import org.apache.kafka.common.Node; import org.apache.kafka.common.PartitionInfo; import org.apache.kafka.common.TopicPartition; @@ -267,18 +268,9 @@ public void shouldThrowOnSendIfProducerGotFenced() { try { producer.send(null); fail("Should have thrown as producer is fenced off"); - } catch (ProducerFencedException e) { } - } - - @Test - public void shouldThrowOnFlushIfProducerGotFenced() { - buildMockProducer(true); - producer.initTransactions(); - producer.fenceProducer(); - try { - producer.flush(); - fail("Should have thrown as producer is fenced off"); - } catch (ProducerFencedException e) { } + } catch (KafkaException e) { + assertTrue("The root cause of the exception should be ProducerFenced", e.getCause() instanceof ProducerFencedException); + } } @Test @@ -636,16 +628,6 @@ public void shouldThrowOnAbortTransactionIfProducerIsClosed() { } catch (IllegalStateException e) { } } - @Test - public void shouldThrowOnCloseIfProducerIsClosed() { - buildMockProducer(true); - producer.close(); - try { - producer.close(); - fail("Should have thrown as producer is already closed"); - } catch (IllegalStateException e) { } - } - @Test public void shouldThrowOnFenceProducerIfProducerIsClosed() { buildMockProducer(true); diff --git a/clients/src/test/java/org/apache/kafka/clients/producer/internals/BufferPoolTest.java b/clients/src/test/java/org/apache/kafka/clients/producer/internals/BufferPoolTest.java index 23fc5411b3cc6..cb622400f5bbf 100644 --- a/clients/src/test/java/org/apache/kafka/clients/producer/internals/BufferPoolTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/producer/internals/BufferPoolTest.java @@ -166,7 +166,7 @@ public void run() { /** * 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. - * And verify that the allocation should finish soon after the maxBlockTimeMs. + * And verify that the allocation attempt finishes soon after the maxBlockTimeMs. */ @Test public void testBlockTimeout() throws Exception { @@ -174,10 +174,10 @@ public void testBlockTimeout() throws Exception { ByteBuffer buffer1 = pool.allocate(1, maxBlockTimeMs); ByteBuffer buffer2 = pool.allocate(1, maxBlockTimeMs); ByteBuffer buffer3 = pool.allocate(1, maxBlockTimeMs); - // First two buffers will be de-allocated within maxBlockTimeMs since the most recent de-allocation + // The first two buffers will be de-allocated within maxBlockTimeMs since the most recent allocation delayedDeallocate(pool, buffer1, maxBlockTimeMs / 2); delayedDeallocate(pool, buffer2, maxBlockTimeMs); - // The third buffer will be de-allocated after maxBlockTimeMs since the most recent de-allocation + // The third buffer will be de-allocated after maxBlockTimeMs since the most recent allocation delayedDeallocate(pool, buffer3, maxBlockTimeMs / 2 * 5); long beginTimeMs = Time.SYSTEM.milliseconds(); @@ -187,9 +187,11 @@ public void testBlockTimeout() throws Exception { } catch (TimeoutException e) { // this is good } - assertTrue("available memory" + pool.availableMemory(), pool.availableMemory() >= 9 && pool.availableMemory() <= 10); - long endTimeMs = Time.SYSTEM.milliseconds(); - assertTrue("Allocation should finish not much later than maxBlockTimeMs", endTimeMs - beginTimeMs < maxBlockTimeMs + 1000); + // Thread scheduling sometimes means that deallocation varies by this point + assertTrue("available memory " + pool.availableMemory(), pool.availableMemory() >= 8 && pool.availableMemory() <= 10); + long durationMs = Time.SYSTEM.milliseconds() - beginTimeMs; + assertTrue("TimeoutException should not throw before maxBlockTimeMs", durationMs >= maxBlockTimeMs); + assertTrue("TimeoutException should throw soon after maxBlockTimeMs", durationMs < maxBlockTimeMs + 1000); } /** @@ -205,7 +207,8 @@ public void testCleanupMemoryAvailabilityWaiterOnBlockTimeout() throws Exception } catch (TimeoutException e) { // this is good } - assertTrue(pool.queued() == 0); + assertEquals(0, pool.queued()); + assertEquals(1, pool.availableMemory()); } /** 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 index 77005b7015277..de4ce5f56e982 100644 --- 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 @@ -64,6 +64,7 @@ import org.apache.kafka.common.utils.MockTime; import org.apache.kafka.test.DelayedReceive; import org.apache.kafka.test.MockSelector; +import org.apache.kafka.test.TestCondition; import org.apache.kafka.test.TestUtils; import org.junit.After; import org.junit.Before; @@ -263,8 +264,8 @@ public void testQuotaMetrics() throws Exception { for (int i = 1; i <= 3; i++) { int throttleTimeMs = 100 * i; ProduceRequest.Builder builder = ProduceRequest.Builder.forCurrentMagic((short) 1, 1000, - Collections.emptyMap()); - ClientRequest request = client.newClientRequest(node.idString(), builder, time.milliseconds(), true, null); + Collections.emptyMap()); + ClientRequest request = client.newClientRequest(node.idString(), builder, time.milliseconds(), true); client.send(request, time.milliseconds()); client.poll(1, time.milliseconds()); ProduceResponse response = produceResponse(tp0, i, Errors.NONE, throttleTimeMs); @@ -527,7 +528,7 @@ public void testCanRetryWithoutIdempotence() throws Exception { @Override public boolean matches(AbstractRequest body) { ProduceRequest request = (ProduceRequest) body; - assertFalse(request.isIdempotent()); + assertFalse(request.hasIdempotentRecords()); return true; } }, produceResponse(tp0, -1L, Errors.TOPIC_AUTHORIZATION_FAILED, 0)); @@ -1155,6 +1156,80 @@ public void testResetOfProducerStateShouldAllowQueuedBatchesToDrain() throws Exc assertEquals(0, transactionManager.sequenceNumber(tp1).longValue()); } + @Test + public void testCloseWithProducerIdReset() throws Exception { + final long producerId = 343434L; + TransactionManager transactionManager = new TransactionManager(); + transactionManager.setProducerIdAndEpoch(new ProducerIdAndEpoch(producerId, (short) 0)); + setupWithTransactionState(transactionManager); + + Metrics m = new Metrics(); + SenderMetricsRegistry senderMetrics = new SenderMetricsRegistry(m); + + Sender sender = new Sender(logContext, client, metadata, this.accumulator, true, MAX_REQUEST_SIZE, ACKS_ALL, 10, + senderMetrics, time, REQUEST_TIMEOUT, 50, transactionManager, apiVersions); + + Future failedResponse = accumulator.append(tp0, time.milliseconds(), "key".getBytes(), + "value".getBytes(), null, null, MAX_BLOCK_TIMEOUT).future; + Future successfulResponse = accumulator.append(tp1, time.milliseconds(), "key".getBytes(), + "value".getBytes(), null, null, MAX_BLOCK_TIMEOUT).future; + sender.run(time.milliseconds()); // connect and send. + + assertEquals(1, client.inFlightRequestCount()); + + Map responses = new LinkedHashMap<>(); + responses.put(tp1, new OffsetAndError(-1, Errors.NOT_LEADER_FOR_PARTITION)); + responses.put(tp0, new OffsetAndError(-1, Errors.OUT_OF_ORDER_SEQUENCE_NUMBER)); + client.respond(produceResponse(responses)); + sender.initiateClose(); // initiate close + sender.run(time.milliseconds()); + assertTrue(failedResponse.isDone()); + assertFalse("Expected transaction state to be reset upon receiving an OutOfOrderSequenceException", transactionManager.hasProducerId()); + + TestUtils.waitForCondition(new TestCondition() { + @Override + public boolean conditionMet() { + prepareInitPidResponse(Errors.NONE, producerId + 1, (short) 1); + sender.run(time.milliseconds()); + return !accumulator.hasUndrained(); + } + }, 5000, "Failed to drain batches"); + } + + @Test + public void testForceCloseWithProducerIdReset() throws Exception { + TransactionManager transactionManager = new TransactionManager(); + transactionManager.setProducerIdAndEpoch(new ProducerIdAndEpoch(1L, (short) 0)); + setupWithTransactionState(transactionManager); + + Metrics m = new Metrics(); + SenderMetricsRegistry senderMetrics = new SenderMetricsRegistry(m); + + Sender sender = new Sender(logContext, client, metadata, this.accumulator, true, MAX_REQUEST_SIZE, ACKS_ALL, 10, + senderMetrics, time, REQUEST_TIMEOUT, 50, transactionManager, apiVersions); + + Future failedResponse = accumulator.append(tp0, time.milliseconds(), "key".getBytes(), + "value".getBytes(), null, null, MAX_BLOCK_TIMEOUT).future; + Future successfulResponse = accumulator.append(tp1, time.milliseconds(), "key".getBytes(), + "value".getBytes(), null, null, MAX_BLOCK_TIMEOUT).future; + sender.run(time.milliseconds()); // connect and send. + + assertEquals(1, client.inFlightRequestCount()); + + Map responses = new LinkedHashMap<>(); + responses.put(tp1, new OffsetAndError(-1, Errors.NOT_LEADER_FOR_PARTITION)); + responses.put(tp0, new OffsetAndError(-1, Errors.OUT_OF_ORDER_SEQUENCE_NUMBER)); + client.respond(produceResponse(responses)); + sender.run(time.milliseconds()); + assertTrue(failedResponse.isDone()); + assertFalse("Expected transaction state to be reset upon receiving an OutOfOrderSequenceException", transactionManager.hasProducerId()); + sender.forceClose(); // initiate force close + sender.run(time.milliseconds()); // this should not block + sender.run(); // run main loop to test forceClose flag + assertTrue("Pending batches are not aborted.", !accumulator.hasUndrained()); + assertTrue(successfulResponse.isDone()); + } + @Test public void testBatchesDrainedWithOldProducerIdShouldFailWithOutOfOrderSequenceOnSubsequentRetry() throws Exception { final long producerId = 343434L; @@ -1513,7 +1588,7 @@ void sendIdempotentProducerResponse(final int expectedSequence, TopicPartition t @Override public boolean matches(AbstractRequest body) { ProduceRequest produceRequest = (ProduceRequest) body; - assertTrue(produceRequest.isIdempotent()); + assertTrue(produceRequest.hasIdempotentRecords()); MemoryRecords records = produceRequest.partitionRecordsOrFail().get(tp0); Iterator batchIterator = records.batches().iterator(); @@ -1542,7 +1617,7 @@ public void testClusterAuthorizationExceptionInProduceRequest() throws Exception client.prepareResponse(new MockClient.RequestMatcher() { @Override public boolean matches(AbstractRequest body) { - return body instanceof ProduceRequest && ((ProduceRequest) body).isIdempotent(); + return body instanceof ProduceRequest && ((ProduceRequest) body).hasIdempotentRecords(); } }, produceResponse(tp0, -1, Errors.CLUSTER_AUTHORIZATION_FAILED, 0)); @@ -1576,7 +1651,7 @@ public void testCancelInFlightRequestAfterFatalError() throws Exception { client.respond(new MockClient.RequestMatcher() { @Override public boolean matches(AbstractRequest body) { - return body instanceof ProduceRequest && ((ProduceRequest) body).isIdempotent(); + return body instanceof ProduceRequest && ((ProduceRequest) body).hasIdempotentRecords(); } }, produceResponse(tp0, -1, Errors.CLUSTER_AUTHORIZATION_FAILED, 0)); @@ -1591,7 +1666,7 @@ public boolean matches(AbstractRequest body) { client.respond(new MockClient.RequestMatcher() { @Override public boolean matches(AbstractRequest body) { - return body instanceof ProduceRequest && ((ProduceRequest) body).isIdempotent(); + return body instanceof ProduceRequest && ((ProduceRequest) body).hasIdempotentRecords(); } }, produceResponse(tp1, 0, Errors.NONE, 0)); sender.run(time.milliseconds()); @@ -1612,7 +1687,7 @@ public void testUnsupportedForMessageFormatInProduceRequest() throws Exception { client.prepareResponse(new MockClient.RequestMatcher() { @Override public boolean matches(AbstractRequest body) { - return body instanceof ProduceRequest && ((ProduceRequest) body).isIdempotent(); + return body instanceof ProduceRequest && ((ProduceRequest) body).hasIdempotentRecords(); } }, produceResponse(tp0, -1, Errors.UNSUPPORTED_FOR_MESSAGE_FORMAT, 0)); @@ -1638,7 +1713,7 @@ public void testUnsupportedVersionInProduceRequest() throws Exception { client.prepareUnsupportedVersionResponse(new MockClient.RequestMatcher() { @Override public boolean matches(AbstractRequest body) { - return body instanceof ProduceRequest && ((ProduceRequest) body).isIdempotent(); + return body instanceof ProduceRequest && ((ProduceRequest) body).hasIdempotentRecords(); } }); diff --git a/clients/src/test/java/org/apache/kafka/clients/producer/internals/TransactionManagerTest.java b/clients/src/test/java/org/apache/kafka/clients/producer/internals/TransactionManagerTest.java index 558ec72109695..c5fab594a00e0 100644 --- a/clients/src/test/java/org/apache/kafka/clients/producer/internals/TransactionManagerTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/producer/internals/TransactionManagerTest.java @@ -125,7 +125,8 @@ public void setup() { Metrics metrics = new Metrics(metricConfig, time); SenderMetricsRegistry senderMetrics = new SenderMetricsRegistry(metrics); - this.accumulator = new RecordAccumulator(logContext, batchSize, 1024 * 1024, CompressionType.NONE, 0L, 0L, metrics, time, apiVersions, transactionManager); + this.accumulator = new RecordAccumulator(logContext, batchSize, 1024 * 1024, CompressionType.NONE, + 0L, 0L, metrics, time, apiVersions, transactionManager); this.sender = new Sender(logContext, this.client, this.metadata, this.accumulator, true, MAX_REQUEST_SIZE, ACKS_ALL, MAX_RETRIES, senderMetrics, this.time, REQUEST_TIMEOUT, 50, transactionManager, apiVersions); this.metadata.update(this.cluster, Collections.emptySet(), time.milliseconds()); @@ -574,6 +575,18 @@ public void testDefaultSequenceNumber() { assertEquals((int) transactionManager.sequenceNumber(tp0), 3); } + @Test + public void testSequenceNumberOverflow() { + TransactionManager transactionManager = new TransactionManager(); + assertEquals((int) transactionManager.sequenceNumber(tp0), 0); + transactionManager.incrementSequenceNumber(tp0, Integer.MAX_VALUE); + assertEquals((int) transactionManager.sequenceNumber(tp0), Integer.MAX_VALUE); + transactionManager.incrementSequenceNumber(tp0, 100); + assertEquals((int) transactionManager.sequenceNumber(tp0), 99); + transactionManager.incrementSequenceNumber(tp0, Integer.MAX_VALUE); + assertEquals((int) transactionManager.sequenceNumber(tp0), 98); + } + @Test public void testProducerIdReset() { TransactionManager transactionManager = new TransactionManager(); @@ -916,13 +929,13 @@ public void testGroupAuthorizationFailureInTxnOffsetCommit() { final String consumerGroupId = "consumer"; final long pid = 13131L; final short epoch = 1; - final TopicPartition tp = new TopicPartition("foo", 0); + final TopicPartition tp1 = new TopicPartition("foo", 0); doInitTransactions(pid, epoch); transactionManager.beginTransaction(); TransactionalRequestResult sendOffsetsResult = transactionManager.sendOffsetsToTransaction( - singletonMap(tp, new OffsetAndMetadata(39L)), consumerGroupId); + singletonMap(tp1, new OffsetAndMetadata(39L)), consumerGroupId); prepareAddOffsetsToTxnResponse(Errors.NONE, consumerGroupId, pid, epoch); sender.run(time.milliseconds()); // AddOffsetsToTxn Handled, TxnOffsetCommit Enqueued @@ -931,7 +944,7 @@ public void testGroupAuthorizationFailureInTxnOffsetCommit() { prepareFindCoordinatorResponse(Errors.NONE, false, CoordinatorType.GROUP, consumerGroupId); sender.run(time.milliseconds()); // FindCoordinator Returned - prepareTxnOffsetCommitResponse(consumerGroupId, pid, epoch, singletonMap(tp, Errors.GROUP_AUTHORIZATION_FAILED)); + prepareTxnOffsetCommitResponse(consumerGroupId, pid, epoch, singletonMap(tp1, Errors.GROUP_AUTHORIZATION_FAILED)); sender.run(time.milliseconds()); // TxnOffsetCommit Handled assertTrue(transactionManager.hasError()); @@ -939,6 +952,7 @@ public void testGroupAuthorizationFailureInTxnOffsetCommit() { assertTrue(sendOffsetsResult.isCompleted()); assertFalse(sendOffsetsResult.isSuccessful()); assertTrue(sendOffsetsResult.error() instanceof GroupAuthorizationException); + assertFalse(transactionManager.hasPendingOffsetCommits()); GroupAuthorizationException exception = (GroupAuthorizationException) sendOffsetsResult.error(); assertEquals(consumerGroupId, exception.groupId()); @@ -1749,7 +1763,16 @@ public void testHandlingOfUnknownTopicPartitionErrorOnAddPartitions() throws Int } @Test - public void testHandlingOfUnknownTopicPartitionErrorOnTxnOffsetCommit() throws InterruptedException { + public void testHandlingOfUnknownTopicPartitionErrorOnTxnOffsetCommit() { + testRetriableErrorInTxnOffsetCommit(Errors.UNKNOWN_TOPIC_OR_PARTITION); + } + + @Test + public void testHandlingOfCoordinatorLoadingErrorOnTxnOffsetCommit() { + testRetriableErrorInTxnOffsetCommit(Errors.COORDINATOR_LOAD_IN_PROGRESS); + } + + private void testRetriableErrorInTxnOffsetCommit(Errors error) { final long pid = 13131L; final short epoch = 1; @@ -1758,6 +1781,7 @@ public void testHandlingOfUnknownTopicPartitionErrorOnTxnOffsetCommit() throws I transactionManager.beginTransaction(); Map offsets = new HashMap<>(); + offsets.put(tp0, new OffsetAndMetadata(1)); offsets.put(tp1, new OffsetAndMetadata(1)); final String consumerGroupId = "myconsumergroup"; @@ -1769,12 +1793,13 @@ public void testHandlingOfUnknownTopicPartitionErrorOnTxnOffsetCommit() throws I assertFalse(addOffsetsResult.isCompleted()); // The request should complete only after the TxnOffsetCommit completes. Map txnOffsetCommitResponse = new HashMap<>(); - txnOffsetCommitResponse.put(tp1, Errors.UNKNOWN_TOPIC_OR_PARTITION); + txnOffsetCommitResponse.put(tp0, Errors.NONE); + txnOffsetCommitResponse.put(tp1, error); prepareFindCoordinatorResponse(Errors.NONE, false, CoordinatorType.GROUP, consumerGroupId); prepareTxnOffsetCommitResponse(consumerGroupId, pid, epoch, txnOffsetCommitResponse); - assertEquals(null, transactionManager.coordinator(CoordinatorType.GROUP)); + assertNull(transactionManager.coordinator(CoordinatorType.GROUP)); sender.run(time.milliseconds()); // try to send TxnOffsetCommitRequest, but find we don't have a group coordinator. sender.run(time.milliseconds()); // send find coordinator for group request assertNotNull(transactionManager.coordinator(CoordinatorType.GROUP)); @@ -1799,7 +1824,7 @@ public void shouldNotAddPartitionsToTransactionWhenTopicAuthorizationFailed() th } @Test - public void shouldNotSendAbortTxnRequestWhenOnlyAddPartitionsRequestFailed() throws Exception { + public void shouldNotSendAbortTxnRequestWhenOnlyAddPartitionsRequestFailed() { final long pid = 13131L; final short epoch = 1; @@ -2395,7 +2420,9 @@ public boolean matches(AbstractRequest body) { }; } - private void prepareAddOffsetsToTxnResponse(Errors error, final String consumerGroupId, final long producerId, + private void prepareAddOffsetsToTxnResponse(final Errors error, + final String consumerGroupId, + final long producerId, final short producerEpoch) { client.prepareResponse(new MockClient.RequestMatcher() { @Override @@ -2445,7 +2472,7 @@ private void doInitTransactions(long pid, short epoch) { private void assertAbortableError(Class cause) { try { - transactionManager.beginTransaction(); + transactionManager.beginCommit(); fail("Should have raised " + cause.getSimpleName()); } catch (KafkaException e) { assertTrue(cause.isAssignableFrom(e.getCause().getClass())); diff --git a/clients/src/test/java/org/apache/kafka/common/acl/AclBindingTest.java b/clients/src/test/java/org/apache/kafka/common/acl/AclBindingTest.java index 6110c48e2b03d..461661fb21ac1 100644 --- a/clients/src/test/java/org/apache/kafka/common/acl/AclBindingTest.java +++ b/clients/src/test/java/org/apache/kafka/common/acl/AclBindingTest.java @@ -16,69 +16,71 @@ */ package org.apache.kafka.common.acl; -import org.apache.kafka.common.resource.Resource; -import org.apache.kafka.common.resource.ResourceFilter; -import org.apache.kafka.common.resource.ResourceNameType; +import org.apache.kafka.common.resource.PatternType; +import org.apache.kafka.common.resource.ResourcePattern; +import org.apache.kafka.common.resource.ResourcePatternFilter; import org.apache.kafka.common.resource.ResourceType; import org.junit.Test; +import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertNotEquals; import static org.junit.Assert.assertNull; import static org.junit.Assert.assertTrue; public class AclBindingTest { private static final AclBinding ACL1 = new AclBinding( - new Resource(ResourceType.TOPIC, "mytopic", ResourceNameType.LITERAL), + new ResourcePattern(ResourceType.TOPIC, "mytopic", PatternType.LITERAL), new AccessControlEntry("User:ANONYMOUS", "", AclOperation.ALL, AclPermissionType.ALLOW)); private static final AclBinding ACL2 = new AclBinding( - new Resource(ResourceType.TOPIC, "mytopic", ResourceNameType.LITERAL), + new ResourcePattern(ResourceType.TOPIC, "mytopic", PatternType.LITERAL), new AccessControlEntry("User:*", "", AclOperation.READ, AclPermissionType.ALLOW)); private static final AclBinding ACL3 = new AclBinding( - new Resource(ResourceType.TOPIC, "mytopic2", ResourceNameType.LITERAL), + new ResourcePattern(ResourceType.TOPIC, "mytopic2", PatternType.LITERAL), new AccessControlEntry("User:ANONYMOUS", "127.0.0.1", AclOperation.READ, AclPermissionType.DENY)); private static final AclBinding UNKNOWN_ACL = new AclBinding( - new Resource(ResourceType.TOPIC, "mytopic2", ResourceNameType.LITERAL), + new ResourcePattern(ResourceType.TOPIC, "mytopic2", PatternType.LITERAL), new AccessControlEntry("User:ANONYMOUS", "127.0.0.1", AclOperation.UNKNOWN, AclPermissionType.DENY)); private static final AclBindingFilter ANY_ANONYMOUS = new AclBindingFilter( - ResourceFilter.ANY, + ResourcePatternFilter.ANY, new AccessControlEntryFilter("User:ANONYMOUS", null, AclOperation.ANY, AclPermissionType.ANY)); private static final AclBindingFilter ANY_DENY = new AclBindingFilter( - ResourceFilter.ANY, + ResourcePatternFilter.ANY, new AccessControlEntryFilter(null, null, AclOperation.ANY, AclPermissionType.DENY)); private static final AclBindingFilter ANY_MYTOPIC = new AclBindingFilter( - new ResourceFilter(ResourceType.TOPIC, "mytopic", ResourceNameType.LITERAL), + new ResourcePatternFilter(ResourceType.TOPIC, "mytopic", PatternType.LITERAL), new AccessControlEntryFilter(null, null, AclOperation.ANY, AclPermissionType.ANY)); @Test - public void testMatching() throws Exception { - assertTrue(ACL1.equals(ACL1)); + public void testMatching() { + assertEquals(ACL1, ACL1); final AclBinding acl1Copy = new AclBinding( - new Resource(ResourceType.TOPIC, "mytopic", ResourceNameType.LITERAL), + new ResourcePattern(ResourceType.TOPIC, "mytopic", PatternType.LITERAL), new AccessControlEntry("User:ANONYMOUS", "", AclOperation.ALL, AclPermissionType.ALLOW)); - assertTrue(ACL1.equals(acl1Copy)); - assertTrue(acl1Copy.equals(ACL1)); - assertTrue(ACL2.equals(ACL2)); - assertFalse(ACL1.equals(ACL2)); - assertFalse(ACL2.equals(ACL1)); + assertEquals(ACL1, acl1Copy); + assertEquals(acl1Copy, ACL1); + assertEquals(ACL2, ACL2); + assertNotEquals(ACL1, ACL2); + assertNotEquals(ACL2, ACL1); assertTrue(AclBindingFilter.ANY.matches(ACL1)); - assertFalse(AclBindingFilter.ANY.equals(ACL1)); + assertNotEquals(AclBindingFilter.ANY, ACL1); assertTrue(AclBindingFilter.ANY.matches(ACL2)); - assertFalse(AclBindingFilter.ANY.equals(ACL2)); + assertNotEquals(AclBindingFilter.ANY, ACL2); assertTrue(AclBindingFilter.ANY.matches(ACL3)); - assertFalse(AclBindingFilter.ANY.equals(ACL3)); - assertTrue(AclBindingFilter.ANY.equals(AclBindingFilter.ANY)); + assertNotEquals(AclBindingFilter.ANY, ACL3); + assertEquals(AclBindingFilter.ANY, AclBindingFilter.ANY); assertTrue(ANY_ANONYMOUS.matches(ACL1)); - assertFalse(ANY_ANONYMOUS.equals(ACL1)); + assertNotEquals(ANY_ANONYMOUS, ACL1); assertFalse(ANY_ANONYMOUS.matches(ACL2)); - assertFalse(ANY_ANONYMOUS.equals(ACL2)); + assertNotEquals(ANY_ANONYMOUS, ACL2); assertTrue(ANY_ANONYMOUS.matches(ACL3)); - assertFalse(ANY_ANONYMOUS.equals(ACL3)); + assertNotEquals(ANY_ANONYMOUS, ACL3); assertFalse(ANY_DENY.matches(ACL1)); assertFalse(ANY_DENY.matches(ACL2)); assertTrue(ANY_DENY.matches(ACL3)); @@ -87,12 +89,12 @@ public void testMatching() throws Exception { assertFalse(ANY_MYTOPIC.matches(ACL3)); assertTrue(ANY_ANONYMOUS.matches(UNKNOWN_ACL)); assertTrue(ANY_DENY.matches(UNKNOWN_ACL)); - assertTrue(UNKNOWN_ACL.equals(UNKNOWN_ACL)); + assertEquals(UNKNOWN_ACL, UNKNOWN_ACL); assertFalse(ANY_MYTOPIC.matches(UNKNOWN_ACL)); } @Test - public void testUnknowns() throws Exception { + public void testUnknowns() { assertFalse(ACL1.isUnknown()); assertFalse(ACL2.isUnknown()); assertFalse(ACL3.isUnknown()); @@ -103,7 +105,7 @@ public void testUnknowns() throws Exception { } @Test - public void testMatchesAtMostOne() throws Exception { + public void testMatchesAtMostOne() { assertNull(ACL1.toFilter().findIndefiniteField()); assertNull(ACL2.toFilter().findIndefiniteField()); assertNull(ACL3.toFilter().findIndefiniteField()); @@ -111,4 +113,29 @@ public void testMatchesAtMostOne() throws Exception { assertFalse(ANY_DENY.matchesAtMostOne()); assertFalse(ANY_MYTOPIC.matchesAtMostOne()); } + + @Test + public void shouldNotThrowOnUnknownPatternType() { + new AclBinding(new ResourcePattern(ResourceType.TOPIC, "foo", PatternType.UNKNOWN), ACL1.entry()); + } + + @Test + public void shouldNotThrowOnUnknownResourceType() { + new AclBinding(new ResourcePattern(ResourceType.UNKNOWN, "foo", PatternType.LITERAL), ACL1.entry()); + } + + @Test(expected = IllegalArgumentException.class) + public void shouldThrowOnMatchPatternType() { + new AclBinding(new ResourcePattern(ResourceType.TOPIC, "foo", PatternType.MATCH), ACL1.entry()); + } + + @Test(expected = IllegalArgumentException.class) + public void shouldThrowOnAnyPatternType() { + new AclBinding(new ResourcePattern(ResourceType.TOPIC, "foo", PatternType.ANY), ACL1.entry()); + } + + @Test(expected = IllegalArgumentException.class) + public void shouldThrowOnAnyResourceType() { + new AclBinding(new ResourcePattern(ResourceType.ANY, "foo", PatternType.LITERAL), ACL1.entry()); + } } diff --git a/clients/src/test/java/org/apache/kafka/common/acl/ResourcePatternFilterTest.java b/clients/src/test/java/org/apache/kafka/common/acl/ResourcePatternFilterTest.java new file mode 100644 index 0000000000000..08d5a63450841 --- /dev/null +++ b/clients/src/test/java/org/apache/kafka/common/acl/ResourcePatternFilterTest.java @@ -0,0 +1,176 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF 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.acl; + +import org.apache.kafka.common.resource.PatternType; +import org.apache.kafka.common.resource.ResourcePattern; +import org.apache.kafka.common.resource.ResourcePatternFilter; +import org.junit.Test; + +import static org.apache.kafka.common.resource.PatternType.LITERAL; +import static org.apache.kafka.common.resource.PatternType.PREFIXED; +import static org.apache.kafka.common.resource.ResourceType.ANY; +import static org.apache.kafka.common.resource.ResourceType.GROUP; +import static org.apache.kafka.common.resource.ResourceType.TOPIC; +import static org.apache.kafka.common.resource.ResourceType.UNKNOWN; +import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertTrue; + +public class ResourcePatternFilterTest { + @Test + public void shouldBeUnknownIfResourceTypeUnknown() { + assertTrue(new ResourcePatternFilter(UNKNOWN, null, PatternType.LITERAL).isUnknown()); + } + + @Test + public void shouldBeUnknownIfPatternTypeUnknown() { + assertTrue(new ResourcePatternFilter(GROUP, null, PatternType.UNKNOWN).isUnknown()); + } + + @Test + public void shouldNotMatchIfDifferentResourceType() { + assertFalse(new ResourcePatternFilter(TOPIC, "Name", LITERAL) + .matches(new ResourcePattern(GROUP, "Name", LITERAL))); + } + + @Test + public void shouldNotMatchIfDifferentName() { + assertFalse(new ResourcePatternFilter(TOPIC, "Different", PREFIXED) + .matches(new ResourcePattern(TOPIC, "Name", PREFIXED))); + } + + @Test + public void shouldNotMatchIfDifferentNameCase() { + assertFalse(new ResourcePatternFilter(TOPIC, "NAME", LITERAL) + .matches(new ResourcePattern(TOPIC, "Name", LITERAL))); + } + + @Test + public void shouldNotMatchIfDifferentPatternType() { + assertFalse(new ResourcePatternFilter(TOPIC, "Name", LITERAL) + .matches(new ResourcePattern(TOPIC, "Name", PREFIXED))); + } + + @Test + public void shouldMatchWhereResourceTypeIsAny() { + assertTrue(new ResourcePatternFilter(ANY, "Name", PREFIXED) + .matches(new ResourcePattern(TOPIC, "Name", PREFIXED))); + } + + @Test + public void shouldMatchWhereResourceNameIsAny() { + assertTrue(new ResourcePatternFilter(TOPIC, null, PREFIXED) + .matches(new ResourcePattern(TOPIC, "Name", PREFIXED))); + } + + @Test + public void shouldMatchWherePatternTypeIsAny() { + assertTrue(new ResourcePatternFilter(TOPIC, null, PatternType.ANY) + .matches(new ResourcePattern(TOPIC, "Name", PREFIXED))); + } + + @Test + public void shouldMatchWherePatternTypeIsMatch() { + assertTrue(new ResourcePatternFilter(TOPIC, null, PatternType.MATCH) + .matches(new ResourcePattern(TOPIC, "Name", PREFIXED))); + } + + @Test + public void shouldMatchLiteralIfExactMatch() { + assertTrue(new ResourcePatternFilter(TOPIC, "Name", LITERAL) + .matches(new ResourcePattern(TOPIC, "Name", LITERAL))); + } + + @Test + public void shouldMatchLiteralIfNameMatchesAndFilterIsOnPatternTypeAny() { + assertTrue(new ResourcePatternFilter(TOPIC, "Name", PatternType.ANY) + .matches(new ResourcePattern(TOPIC, "Name", LITERAL))); + } + + @Test + public void shouldMatchLiteralIfNameMatchesAndFilterIsOnPatternTypeMatch() { + assertTrue(new ResourcePatternFilter(TOPIC, "Name", PatternType.MATCH) + .matches(new ResourcePattern(TOPIC, "Name", LITERAL))); + } + + @Test + public void shouldNotMatchLiteralIfNamePrefixed() { + assertFalse(new ResourcePatternFilter(TOPIC, "Name-something", PatternType.MATCH) + .matches(new ResourcePattern(TOPIC, "Name", LITERAL))); + } + + @Test + public void shouldMatchLiteralWildcardIfExactMatch() { + assertTrue(new ResourcePatternFilter(TOPIC, "*", LITERAL) + .matches(new ResourcePattern(TOPIC, "*", LITERAL))); + } + + @Test + public void shouldNotMatchLiteralWildcardAgainstOtherName() { + assertFalse(new ResourcePatternFilter(TOPIC, "Name", LITERAL) + .matches(new ResourcePattern(TOPIC, "*", LITERAL))); + } + + @Test + public void shouldNotMatchLiteralWildcardTheWayAround() { + assertFalse(new ResourcePatternFilter(TOPIC, "*", LITERAL) + .matches(new ResourcePattern(TOPIC, "Name", LITERAL))); + } + + @Test + public void shouldNotMatchLiteralWildcardIfFilterHasPatternTypeOfAny() { + assertFalse(new ResourcePatternFilter(TOPIC, "Name", PatternType.ANY) + .matches(new ResourcePattern(TOPIC, "*", LITERAL))); + } + + @Test + public void shouldMatchLiteralWildcardIfFilterHasPatternTypeOfMatch() { + assertTrue(new ResourcePatternFilter(TOPIC, "Name", PatternType.MATCH) + .matches(new ResourcePattern(TOPIC, "*", LITERAL))); + } + + @Test + public void shouldMatchPrefixedIfExactMatch() { + assertTrue(new ResourcePatternFilter(TOPIC, "Name", PREFIXED) + .matches(new ResourcePattern(TOPIC, "Name", PREFIXED))); + } + + @Test + public void shouldNotMatchIfBothPrefixedAndFilterIsPrefixOfResource() { + assertFalse(new ResourcePatternFilter(TOPIC, "Name", PREFIXED) + .matches(new ResourcePattern(TOPIC, "Name-something", PREFIXED))); + } + + @Test + public void shouldNotMatchIfBothPrefixedAndResourceIsPrefixOfFilter() { + assertFalse(new ResourcePatternFilter(TOPIC, "Name-something", PREFIXED) + .matches(new ResourcePattern(TOPIC, "Name", PREFIXED))); + } + + @Test + public void shouldNotMatchPrefixedIfNamePrefixedAnyFilterTypeIsAny() { + assertFalse(new ResourcePatternFilter(TOPIC, "Name-something", PatternType.ANY) + .matches(new ResourcePattern(TOPIC, "Name", PREFIXED))); + } + + @Test + public void shouldMatchPrefixedIfNamePrefixedAnyFilterTypeIsMatch() { + assertTrue(new ResourcePatternFilter(TOPIC, "Name-something", PatternType.MATCH) + .matches(new ResourcePattern(TOPIC, "Name", PREFIXED))); + } +} \ No newline at end of file diff --git a/clients/src/test/java/org/apache/kafka/common/acl/ResourcePatternTest.java b/clients/src/test/java/org/apache/kafka/common/acl/ResourcePatternTest.java new file mode 100644 index 0000000000000..d3538e012e334 --- /dev/null +++ b/clients/src/test/java/org/apache/kafka/common/acl/ResourcePatternTest.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.common.acl; + +import org.apache.kafka.common.resource.PatternType; +import org.apache.kafka.common.resource.ResourcePattern; +import org.apache.kafka.common.resource.ResourceType; +import org.junit.Test; + +public class ResourcePatternTest { + @Test(expected = IllegalArgumentException.class) + public void shouldThrowIfResourceTypeIsAny() { + new ResourcePattern(ResourceType.ANY, "name", PatternType.LITERAL); + } + + @Test(expected = IllegalArgumentException.class) + public void shouldThrowIfPatternTypeIsMatch() { + new ResourcePattern(ResourceType.TOPIC, "name", PatternType.MATCH); + } + + @Test(expected = IllegalArgumentException.class) + public void shouldThrowIfPatternTypeIsAny() { + new ResourcePattern(ResourceType.TOPIC, "name", PatternType.ANY); + } + + @Test(expected = NullPointerException.class) + public void shouldThrowIfResourceNameIsNull() { + new ResourcePattern(ResourceType.TOPIC, null, PatternType.ANY); + } +} \ No newline at end of file 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 index 071deed47e422..5762adf719548 100644 --- a/clients/src/test/java/org/apache/kafka/common/config/AbstractConfigTest.java +++ b/clients/src/test/java/org/apache/kafka/common/config/AbstractConfigTest.java @@ -263,7 +263,7 @@ public RestrictedClassLoader() { @Override protected Class findClass(String name) throws ClassNotFoundException { if (name.equals(ClassTestConfig.DEFAULT_CLASS.getName()) || name.equals(ClassTestConfig.RESTRICTED_CLASS.getName())) - return null; + throw new ClassNotFoundException(); else return ClassTestConfig.class.getClassLoader().loadClass(name); } 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 affa5dd44367d..70700422a74f7 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 @@ -639,6 +639,29 @@ public void testConvertValueToStringNestedClass() throws ClassNotFoundException assertEquals(NestedClass.class, Class.forName(actual)); } + @Test + public void testClassWithAlias() { + final String alias = "PluginAlias"; + ClassLoader originalClassLoader = Thread.currentThread().getContextClassLoader(); + try { + // Could try to use the Plugins class from Connect here, but this should simulate enough + // of the aliasing logic to suffice for this test. + Thread.currentThread().setContextClassLoader(new ClassLoader(originalClassLoader) { + @Override + public Class loadClass(String name, boolean resolve) throws ClassNotFoundException { + if (alias.equals(name)) { + return NestedClass.class; + } else { + return super.loadClass(name, resolve); + } + } + }); + ConfigDef.parseType("Test config", alias, Type.CLASS); + } finally { + Thread.currentThread().setContextClassLoader(originalClassLoader); + } + } + private class NestedClass { } } diff --git a/clients/src/test/java/org/apache/kafka/common/config/ConfigResourceTest.java b/clients/src/test/java/org/apache/kafka/common/config/ConfigResourceTest.java new file mode 100644 index 0000000000000..73effeead48d3 --- /dev/null +++ b/clients/src/test/java/org/apache/kafka/common/config/ConfigResourceTest.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.common.config; + +import org.junit.Test; + +import java.util.Arrays; + +import static org.junit.Assert.assertEquals; + +public class ConfigResourceTest { + @Test + public void shouldGetTypeFromId() { + assertEquals(ConfigResource.Type.TOPIC, ConfigResource.Type.forId((byte) 2)); + assertEquals(ConfigResource.Type.BROKER, ConfigResource.Type.forId((byte) 4)); + } + + @Test + public void shouldReturnUnknownForUnknownCode() { + assertEquals(ConfigResource.Type.UNKNOWN, ConfigResource.Type.forId((byte) -1)); + assertEquals(ConfigResource.Type.UNKNOWN, ConfigResource.Type.forId((byte) 0)); + assertEquals(ConfigResource.Type.UNKNOWN, ConfigResource.Type.forId((byte) 1)); + } + + @Test + public void shouldRoundTripEveryType() { + Arrays.stream(ConfigResource.Type.values()).forEach(type -> + assertEquals(type.toString(), type, ConfigResource.Type.forId(type.id()))); + } +} \ No newline at end of file diff --git a/clients/src/test/java/org/apache/kafka/common/config/ConfigTransformerTest.java b/clients/src/test/java/org/apache/kafka/common/config/ConfigTransformerTest.java index 7bc74f36e9b86..12c6b1f4a2785 100644 --- a/clients/src/test/java/org/apache/kafka/common/config/ConfigTransformerTest.java +++ b/clients/src/test/java/org/apache/kafka/common/config/ConfigTransformerTest.java @@ -16,6 +16,7 @@ */ package org.apache.kafka.common.config; +import org.apache.kafka.common.config.provider.ConfigProvider; import org.junit.Before; import org.junit.Test; @@ -25,6 +26,7 @@ import java.util.Set; import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertNull; import static org.junit.Assert.assertTrue; public class ConfigTransformerTest { @@ -36,6 +38,7 @@ public class ConfigTransformerTest { public static final String TEST_PATH = "testPath"; public static final String TEST_RESULT = "testResult"; public static final String TEST_RESULT_WITH_TTL = "testResultWithTTL"; + public static final String TEST_RESULT_NO_PATH = "testResultNoPath"; private ConfigTransformer configTransformer; @@ -83,6 +86,31 @@ public void testSingleLevelOfIndirection() throws Exception { assertEquals("${test:testPath:testResult}", data.get(MY_KEY)); } + @Test + public void testReplaceVariableNoPath() throws Exception { + ConfigTransformerResult result = configTransformer.transform(Collections.singletonMap(MY_KEY, "${test:testKey}")); + Map data = result.data(); + Map ttls = result.ttls(); + assertEquals(TEST_RESULT_NO_PATH, data.get(MY_KEY)); + assertTrue(ttls.isEmpty()); + } + + @Test + public void testReplaceMultipleVariablesWithoutPathInValue() throws Exception { + ConfigTransformerResult result = configTransformer.transform(Collections.singletonMap(MY_KEY, "first ${test:testKey}; second ${test:testKey}")); + Map data = result.data(); + assertEquals("first testResultNoPath; second testResultNoPath", data.get(MY_KEY)); + } + + @Test + public void testNullConfigValue() throws Exception { + ConfigTransformerResult result = configTransformer.transform(Collections.singletonMap(MY_KEY, null)); + Map data = result.data(); + Map ttls = result.ttls(); + assertNull(data.get(MY_KEY)); + assertTrue(ttls.isEmpty()); + } + public static class TestConfigProvider implements ConfigProvider { public void configure(Map configs) { @@ -95,7 +123,7 @@ public ConfigData get(String path) { public ConfigData get(String path, Set keys) { Map data = new HashMap<>(); Long ttl = null; - if (path.equals(TEST_PATH)) { + if (TEST_PATH.equals(path)) { if (keys.contains(TEST_KEY)) { data.put(TEST_KEY, TEST_RESULT); } @@ -106,6 +134,10 @@ public ConfigData get(String path, Set keys) { if (keys.contains(TEST_INDIRECTION)) { data.put(TEST_INDIRECTION, "${test:testPath:testResult}"); } + } else { + if (keys.contains(TEST_KEY)) { + data.put(TEST_KEY, TEST_RESULT_NO_PATH); + } } return new ConfigData(data, ttl); } diff --git a/clients/src/test/java/org/apache/kafka/common/config/FileConfigProviderTest.java b/clients/src/test/java/org/apache/kafka/common/config/provider/FileConfigProviderTest.java similarity index 97% rename from clients/src/test/java/org/apache/kafka/common/config/FileConfigProviderTest.java rename to clients/src/test/java/org/apache/kafka/common/config/provider/FileConfigProviderTest.java index 9157e38045686..b2c791afebeaf 100644 --- a/clients/src/test/java/org/apache/kafka/common/config/FileConfigProviderTest.java +++ b/clients/src/test/java/org/apache/kafka/common/config/provider/FileConfigProviderTest.java @@ -14,8 +14,9 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.kafka.common.config; +package org.apache.kafka.common.config.provider; +import org.apache.kafka.common.config.ConfigData; import org.junit.Before; import org.junit.Test; 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 6acc39d35a677..6ad48331fdeef 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 @@ -16,6 +16,8 @@ */ package org.apache.kafka.common.metrics; +import static java.util.Collections.emptyList; +import static java.util.Collections.singletonList; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertFalse; import static org.junit.Assert.assertNotNull; @@ -26,13 +28,16 @@ import java.util.Arrays; import java.util.Collections; import java.util.Deque; +import java.util.List; import java.util.HashMap; import java.util.Map; import java.util.Random; import java.util.concurrent.ConcurrentLinkedDeque; import java.util.concurrent.ExecutorService; import java.util.concurrent.Executors; +import java.util.concurrent.Future; import java.util.concurrent.TimeUnit; + import java.util.concurrent.atomic.AtomicBoolean; import org.apache.kafka.common.Metric; @@ -54,9 +59,12 @@ import org.junit.After; import org.junit.Before; import org.junit.Test; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; @SuppressWarnings("deprecation") public class MetricsTest { + private static final Logger log = LoggerFactory.getLogger(MetricsTest.class); private static final double EPS = 0.000001; private MockTime time = new MockTime(); @@ -191,6 +199,20 @@ public void testBadSensorHierarchy() { metrics.sensor("gc", c1, c2); // should fail } + @Test + public void testRemoveChildSensor() { + final Metrics metrics = new Metrics(); + + final Sensor parent = metrics.sensor("parent"); + final Sensor child = metrics.sensor("child", parent); + + assertEquals(singletonList(child), metrics.childrenSensors().get(parent)); + + metrics.removeSensor("child"); + + assertEquals(emptyList(), metrics.childrenSensors().get(parent)); + } + @Test public void testRemoveSensor() { int size = metrics.metrics().size(); @@ -459,8 +481,12 @@ public void testRateWindowing() throws Exception { Sensor s = metrics.sensor("test.sensor", cfg); MetricName rateMetricName = metrics.metricName("test.rate", "grp1"); MetricName totalMetricName = metrics.metricName("test.total", "grp1"); + MetricName countRateMetricName = metrics.metricName("test.count.rate", "grp1"); + MetricName countTotalMetricName = metrics.metricName("test.count.total", "grp1"); s.add(new Meter(TimeUnit.SECONDS, rateMetricName, totalMetricName)); - KafkaMetric totalMetric = metrics.metrics().get(metrics.metricName("test.total", "grp1")); + s.add(new Meter(TimeUnit.SECONDS, new Count(), countRateMetricName, countTotalMetricName)); + KafkaMetric totalMetric = metrics.metrics().get(totalMetricName); + KafkaMetric countTotalMetric = metrics.metrics().get(countTotalMetricName); int sum = 0; int count = cfg.samples() - 1; @@ -478,11 +504,21 @@ public void testRateWindowing() throws Exception { // prior to any time passing double elapsedSecs = (cfg.timeWindowMs() * (cfg.samples() - 1) + cfg.timeWindowMs() / 2) / 1000.0; - KafkaMetric rateMetric = metrics.metrics().get(metrics.metricName("test.rate", "grp1")); + KafkaMetric rateMetric = metrics.metrics().get(rateMetricName); + KafkaMetric countRateMetric = metrics.metrics().get(countRateMetricName); assertEquals("Rate(0...2) = 2.666", sum / elapsedSecs, rateMetric.value(), EPS); + assertEquals("Count rate(0...2) = 0.02666", count / elapsedSecs, countRateMetric.value(), EPS); assertEquals("Elapsed Time = 75 seconds", elapsedSecs, ((Rate) rateMetric.measurable()).windowSize(cfg, time.milliseconds()) / 1000, EPS); assertEquals(sum, totalMetric.value(), EPS); + assertEquals(count, countTotalMetric.value(), EPS); + + // Verify that rates are expired, but total is cumulative + time.sleep(cfg.timeWindowMs() * cfg.samples()); + assertEquals(0, rateMetric.value(), EPS); + assertEquals(0, countRateMetric.value(), EPS); + assertEquals(sum, totalMetric.value(), EPS); + assertEquals(count, countTotalMetric.value(), EPS); } public static class ConstantMeasurable implements Measurable { @@ -604,8 +640,12 @@ public void testMetricInstances() { } } + /** + * Verifies that concurrent sensor add, remove, updates and read don't result + * in errors or deadlock. + */ @Test - public void testConcurrentAccess() throws Exception { + public void testConcurrentReadUpdate() throws Exception { final Random random = new Random(); final Deque sensors = new ConcurrentLinkedDeque<>(); metrics = new Metrics(new MockTime(10)); @@ -613,16 +653,8 @@ public void testConcurrentAccess() throws Exception { final AtomicBoolean alive = new AtomicBoolean(true); executorService = Executors.newSingleThreadExecutor(); - executorService.submit(new Runnable() { - @Override - public void run() { - while (alive.get()) { - for (Sensor sensor : sensors) { - sensor.record(random.nextInt(10000)); - } - } - } - }); + executorService.submit(new ConcurrentMetricOperation(alive, "record", + () -> sensors.forEach(sensor -> sensor.record(random.nextInt(10000))))); for (int i = 0; i < 10000; i++) { if (sensors.size() > 5) { @@ -640,6 +672,97 @@ public void run() { alive.set(false); } + /** + * Verifies that concurrent sensor add, remove, updates and read with a metrics reporter + * that synchronizes on every reporter method doesn't result in errors or deadlock. + */ + @Test + public void testConcurrentReadUpdateReport() throws Exception { + + class LockingReporter implements MetricsReporter { + Map activeMetrics = new HashMap<>(); + @Override + public synchronized void init(List metrics) { + } + + @Override + public synchronized void metricChange(KafkaMetric metric) { + activeMetrics.put(metric.metricName(), metric); + } + + @Override + public synchronized void metricRemoval(KafkaMetric metric) { + activeMetrics.remove(metric.metricName(), metric); + } + + @Override + public synchronized void close() { + } + + @Override + public void configure(Map configs) { + } + + synchronized void processMetrics() { + for (KafkaMetric metric : activeMetrics.values()) { + assertNotNull("Invalid metric value", metric.metricValue()); + } + } + } + + final LockingReporter reporter = new LockingReporter(); + this.metrics.close(); + this.metrics = new Metrics(config, Arrays.asList((MetricsReporter) reporter), new MockTime(10), true); + final Deque sensors = new ConcurrentLinkedDeque<>(); + SensorCreator sensorCreator = new SensorCreator(metrics); + + final Random random = new Random(); + final AtomicBoolean alive = new AtomicBoolean(true); + executorService = Executors.newFixedThreadPool(3); + + Future writeFuture = executorService.submit(new ConcurrentMetricOperation(alive, "record", + () -> sensors.forEach(sensor -> sensor.record(random.nextInt(10000))))); + Future readFuture = executorService.submit(new ConcurrentMetricOperation(alive, "read", + () -> sensors.forEach(sensor -> sensor.metrics().forEach(metric -> + assertNotNull("Invalid metric value", metric.metricValue()))))); + Future reportFuture = executorService.submit(new ConcurrentMetricOperation(alive, "report", + () -> reporter.processMetrics())); + + for (int i = 0; i < 10000; i++) { + if (sensors.size() > 10) { + Sensor sensor = random.nextBoolean() ? sensors.removeFirst() : sensors.removeLast(); + metrics.removeSensor(sensor.name()); + } + StatType statType = StatType.forId(random.nextInt(StatType.values().length)); + sensors.add(sensorCreator.createSensor(statType, i)); + } + assertFalse("Read failed", readFuture.isDone()); + assertFalse("Write failed", writeFuture.isDone()); + assertFalse("Report failed", reportFuture.isDone()); + + alive.set(false); + } + + private class ConcurrentMetricOperation implements Runnable { + private final AtomicBoolean alive; + private final String opName; + private final Runnable op; + ConcurrentMetricOperation(AtomicBoolean alive, String opName, Runnable op) { + this.alive = alive; + this.opName = opName; + this.op = op; + } + public void run() { + try { + while (alive.get()) { + op.run(); + } + } catch (Throwable t) { + log.error("Metric {} failed with exception", opName, t); + } + } + } + enum StatType { AVG(0), TOTAL(1), @@ -676,7 +799,7 @@ private static class SensorCreator { } private Sensor createSensor(StatType statType, int index) { - Sensor sensor = metrics.sensor("kafka.requests"); + Sensor sensor = metrics.sensor("kafka.requests." + index); Map tags = Collections.singletonMap("tag", "tag" + index); switch (statType) { case AVG: diff --git a/clients/src/test/java/org/apache/kafka/common/network/NioEchoServer.java b/clients/src/test/java/org/apache/kafka/common/network/NioEchoServer.java index 2ce9671736ebb..64b7e4e679225 100644 --- a/clients/src/test/java/org/apache/kafka/common/network/NioEchoServer.java +++ b/clients/src/test/java/org/apache/kafka/common/network/NioEchoServer.java @@ -64,7 +64,8 @@ public class NioEchoServer extends Thread { private volatile WritableByteChannel outputChannel; private final CredentialCache credentialCache; private final Metrics metrics; - private int numSent = 0; + private volatile int numSent = 0; + private volatile boolean closeKafkaChannels; private final DelegationTokenCache tokenCache; public NioEchoServer(ListenerName listenerName, SecurityProtocol securityProtocol, AbstractConfig config, @@ -119,7 +120,7 @@ public void verifyAuthenticationMetrics(int successfulAuthentications, final int waitForMetric("failed-authentication", failedAuthentications); } - private void waitForMetric(String name, final double expectedValue) throws InterruptedException { + public void waitForMetric(String name, final double expectedValue) throws InterruptedException { final String totalName = name + "-total"; final String rateName = name + "-rate"; if (expectedValue == 0.0) { @@ -155,6 +156,10 @@ public void run() { } newChannels.clear(); } + if (closeKafkaChannels) { + for (KafkaChannel channel : selector.channels()) + selector.close(channel.id()); + } List completedReceives = selector.completedReceives(); for (NetworkReceive rcv : completedReceives) { @@ -174,7 +179,6 @@ public void run() { selector.unmute(send.destination()); numSent += 1; } - } } catch (IOException e) { // ignore @@ -208,15 +212,28 @@ public Selector selector() { return selector; } - public void closeConnections() throws IOException { - for (SocketChannel channel : socketChannels) + public void closeKafkaChannels() throws IOException { + closeKafkaChannels = true; + selector.wakeup(); + try { + TestUtils.waitForCondition(() -> selector.channels().isEmpty(), "Channels not closed"); + } catch (InterruptedException e) { + throw new RuntimeException(e); + } finally { + closeKafkaChannels = false; + } + } + + public void closeSocketChannels() throws IOException { + for (SocketChannel channel : socketChannels) { channel.close(); + } socketChannels.clear(); } public void close() throws IOException, InterruptedException { this.serverSocketChannel.close(); - closeConnections(); + closeSocketChannels(); acceptorThread.interrupt(); acceptorThread.join(); interrupt(); 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 8ce8d5043bee7..a4a8efdd37d17 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 @@ -17,8 +17,10 @@ package org.apache.kafka.common.network; import org.apache.kafka.common.KafkaException; +import org.apache.kafka.common.MetricName; import org.apache.kafka.common.memory.MemoryPool; import org.apache.kafka.common.memory.SimpleMemoryPool; +import org.apache.kafka.common.metrics.KafkaMetric; import org.apache.kafka.common.metrics.Metrics; import org.apache.kafka.common.security.auth.SecurityProtocol; import org.apache.kafka.common.utils.LogContext; @@ -49,6 +51,7 @@ import java.util.Map; import java.util.Random; import java.util.Set; +import java.util.Optional; import static org.easymock.EasyMock.createControl; import static org.easymock.EasyMock.expect; @@ -360,6 +363,19 @@ public void testCloseOldestConnection() throws Exception { assertEquals(ChannelState.EXPIRED, selector.disconnected().get(id)); } + @Test + public void testIdleExpiryWithoutReadyKeys() throws IOException { + String id = "0"; + selector.connect(id, new InetSocketAddress("localhost", server.port), BUFFER_SIZE, BUFFER_SIZE); + KafkaChannel channel = selector.channel(id); + channel.selectionKey().interestOps(0); + + time.sleep(6000); // The max idle time is 5000ms + selector.poll(0); + assertTrue("The idle connection should have been closed", selector.disconnected().containsKey(id)); + assertEquals(ChannelState.EXPIRED, selector.disconnected().get(id)); + } + @Test public void testImmediatelyConnectedCleaned() throws Exception { Metrics metrics = new Metrics(); // new metrics object to avoid metric registration conflicts @@ -580,6 +596,75 @@ public void testConnectDisconnectDuringInSinglePoll() throws Exception { control.verify(); } + @Test + public void testOutboundConnectionsCountInConnectionCreationMetric() throws Exception { + // create connections + int expectedConnections = 5; + InetSocketAddress addr = new InetSocketAddress("localhost", server.port); + for (int i = 0; i < expectedConnections; i++) + connect(Integer.toString(i), addr); + + // Poll continuously, as we cannot guarantee that the first call will see all connections + int seenConnections = 0; + for (int i = 0; i < 10; i++) { + selector.poll(100L); + seenConnections += selector.connected().size(); + if (seenConnections == expectedConnections) + break; + } + + assertEquals((double) expectedConnections, getMetric("connection-creation-total").metricValue()); + assertEquals((double) expectedConnections, getMetric("connection-count").metricValue()); + } + + @Test + public void testInboundConnectionsCountInConnectionCreationMetric() throws Exception { + int conns = 5; + + try (ServerSocketChannel ss = ServerSocketChannel.open()) { + ss.bind(new InetSocketAddress(0)); + InetSocketAddress serverAddress = (InetSocketAddress) ss.getLocalAddress(); + + for (int i = 0; i < conns; i++) { + Thread sender = createSender(serverAddress, randomPayload(1)); + sender.start(); + SocketChannel channel = ss.accept(); + channel.configureBlocking(false); + + selector.register(Integer.toString(i), channel); + } + } + + assertEquals((double) conns, getMetric("connection-creation-total").metricValue()); + assertEquals((double) conns, getMetric("connection-count").metricValue()); + } + + @Test + public void testMetricsCleanupOnSelectorClose() throws Exception { + Metrics metrics = new Metrics(); + Selector selector = new Selector(5000, metrics, time, "MetricGroup", channelBuilder, new LogContext()) { + @Override + public void close(String id) { + throw new RuntimeException(); + } + }; + assertTrue(metrics.metrics().size() > 1); + String id = "0"; + selector.connect(id, new InetSocketAddress("localhost", server.port), BUFFER_SIZE, BUFFER_SIZE); + + // Close the selector and ensure a RuntimeException has been throw + try { + selector.close(); + fail(); + } catch (RuntimeException e) { + // Expected + } + + // We should only have one remaining metric for kafka-metrics-count, which is a global metric + assertEquals(1, metrics.metrics().size()); + } + + private String blockingRequest(String node, String s) throws IOException { selector.send(createSend(node, s)); selector.poll(1000L); @@ -675,4 +760,13 @@ else if (obj instanceof Map) assertTrue("Field not empty: " + field + " " + obj, ((Map) obj).isEmpty()); } + private KafkaMetric getMetric(String name) throws Exception { + Optional> metric = metrics.metrics().entrySet().stream() + .filter(entry -> entry.getKey().name().equals(name)) + .findFirst(); + if (!metric.isPresent()) + throw new Exception(String.format("Could not find metric called %s", name)); + + return metric.get().getValue(); + } } 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 index 1d78e5aa8e1c6..1f9739bf762d0 100644 --- a/clients/src/test/java/org/apache/kafka/common/network/SslSelectorTest.java +++ b/clients/src/test/java/org/apache/kafka/common/network/SslSelectorTest.java @@ -42,6 +42,9 @@ import java.util.HashMap; import java.util.List; import java.util.Map; +import java.util.Set; +import java.util.concurrent.atomic.AtomicInteger; +import java.util.function.Consumer; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertFalse; @@ -96,6 +99,13 @@ public void testDisconnectWithIntermediateBufferedBytes() throws Exception { connect(node, new InetSocketAddress("localhost", server.port)); selector.send(createSend(node, request)); + waitForBytesBuffered(selector, node); + + selector.close(node); + verifySelectorEmpty(); + } + + private void waitForBytesBuffered(Selector selector, String node) throws Exception { TestUtils.waitForCondition(new TestCondition() { @Override public boolean conditionMet() { @@ -107,8 +117,72 @@ public boolean conditionMet() { } } }, 2000L, "Failed to reach socket state with bytes buffered"); + } - selector.close(node); + @Test + public void testBytesBufferedChannelWithNoIncomingBytes() throws Exception { + verifyNoUnnecessaryPollWithBytesBuffered(key -> + key.interestOps(key.interestOps() & ~SelectionKey.OP_READ)); + } + + @Test + public void testBytesBufferedChannelAfterMute() throws Exception { + verifyNoUnnecessaryPollWithBytesBuffered(key -> ((KafkaChannel) key.attachment()).mute()); + } + + private void verifyNoUnnecessaryPollWithBytesBuffered(Consumer disableRead) + throws Exception { + this.selector.close(); + + String node1 = "1"; + String node2 = "2"; + final AtomicInteger node1Polls = new AtomicInteger(); + + this.channelBuilder = new TestSslChannelBuilder(Mode.CLIENT); + this.channelBuilder.configure(sslClientConfigs); + this.selector = new Selector(5000, metrics, time, "MetricGroup", channelBuilder, new LogContext()) { + @Override + void pollSelectionKeys(Set selectionKeys, boolean isImmediatelyConnected, long currentTimeNanos) { + for (SelectionKey key : selectionKeys) { + KafkaChannel channel = (KafkaChannel) key.attachment(); + if (channel != null && channel.id().equals(node1)) + node1Polls.incrementAndGet(); + } + super.pollSelectionKeys(selectionKeys, isImmediatelyConnected, currentTimeNanos); + } + }; + + // Get node1 into bytes buffered state and then disable read on the socket. + // Truncate the read buffers to ensure that there is buffered data, but not enough to make progress. + int largeRequestSize = 100 * 1024; + connect(node1, new InetSocketAddress("localhost", server.port)); + selector.send(createSend(node1, TestUtils.randomString(largeRequestSize))); + waitForBytesBuffered(selector, node1); + TestSslChannelBuilder.TestSslTransportLayer.transportLayers.get(node1).truncateReadBuffer(); + disableRead.accept(selector.channel(node1).selectionKey()); + + // Clear poll count and count the polls from now on + node1Polls.set(0); + + // Process sends and receives on node2. Test verifies that we don't process node1 + // unnecessarily on each of these polls. + connect(node2, new InetSocketAddress("localhost", server.port)); + int received = 0; + String request = TestUtils.randomString(10); + selector.send(createSend(node2, request)); + while (received < 100) { + received += selector.completedReceives().size(); + if (!selector.completedSends().isEmpty()) { + selector.send(createSend(node2, request)); + } + selector.poll(5); + } + + // Verify that pollSelectionKeys was invoked once to process buffered data + // but not again since there isn't sufficient data to process. + assertEquals(1, node1Polls.get()); + selector.close(node1); + selector.close(node2); verifySelectorEmpty(); } @@ -244,7 +318,6 @@ protected SslTransportLayer buildTransportLayer(SslFactory sslFactory, String id SocketChannel socketChannel = (SocketChannel) key.channel(); SSLEngine sslEngine = sslFactory.createSslEngine(host, socketChannel.socket().getPort()); TestSslTransportLayer transportLayer = new TestSslTransportLayer(id, key, sslEngine); - transportLayer.startHandshake(); return transportLayer; } @@ -252,22 +325,33 @@ protected SslTransportLayer buildTransportLayer(SslFactory sslFactory, String id * TestSslTransportLayer will read from socket once every two tries. This increases * the chance that there will be bytes buffered in the transport layer after read(). */ - class TestSslTransportLayer extends SslTransportLayer { + static class TestSslTransportLayer extends SslTransportLayer { + static Map transportLayers = new HashMap<>(); boolean muteSocket = false; public TestSslTransportLayer(String channelId, SelectionKey key, SSLEngine sslEngine) throws IOException { super(channelId, key, sslEngine); + transportLayers.put(channelId, this); } @Override protected int readFromSocketChannel() throws IOException { if (muteSocket) { - muteSocket = false; + if ((selectionKey().interestOps() & SelectionKey.OP_READ) != 0) + muteSocket = false; return 0; } muteSocket = true; return super.readFromSocketChannel(); } + + // Leave one byte in network read buffer so that some buffered bytes are present, + // but not enough to make progress on a read. + void truncateReadBuffer() throws Exception { + netReadBuffer().position(1); + appReadBuffer().position(0); + muteSocket = true; + } } } 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 index 1f62c10bd51ce..ca80dd960838e 100644 --- a/clients/src/test/java/org/apache/kafka/common/network/SslTransportLayerTest.java +++ b/clients/src/test/java/org/apache/kafka/common/network/SslTransportLayerTest.java @@ -556,6 +556,27 @@ public void testUnsupportedCiphers() throws Exception { server.verifyAuthenticationMetrics(0, 1); } + @Test + public void testServerRequestMetrics() throws Exception { + String node = "0"; + server = createEchoServer(SecurityProtocol.SSL); + createSelector(sslClientConfigs, 16384, 16384, 16384); + InetSocketAddress addr = new InetSocketAddress("localhost", server.port()); + selector.connect(node, addr, 102400, 102400); + NetworkTestUtils.waitForChannelReady(selector, node); + int messageSize = 1024 * 1024; + String message = TestUtils.randomString(messageSize); + selector.send(new NetworkSend(node, ByteBuffer.wrap(message.getBytes()))); + while (selector.completedReceives().isEmpty()) { + selector.poll(100L); + } + int totalBytes = messageSize + 4; // including 4-byte size + server.waitForMetric("incoming-byte", totalBytes); + server.waitForMetric("outgoing-byte", totalBytes); + server.waitForMetric("request", 1); + server.waitForMetric("response", 1); + } + /** * selector.poll() should be able to fetch more data than netReadBuffer from the socket. */ @@ -699,7 +720,8 @@ public boolean conditionMet() { */ @Test public void testIOExceptionsDuringHandshakeRead() throws Exception { - testIOExceptionsDuringHandshake(true, false); + server = createEchoServer(SecurityProtocol.SSL); + testIOExceptionsDuringHandshake(FailureAction.THROW_IO_EXCEPTION, FailureAction.NO_OP); } /** @@ -707,20 +729,60 @@ public void testIOExceptionsDuringHandshakeRead() throws Exception { */ @Test public void testIOExceptionsDuringHandshakeWrite() throws Exception { - testIOExceptionsDuringHandshake(false, true); + server = createEchoServer(SecurityProtocol.SSL); + testIOExceptionsDuringHandshake(FailureAction.NO_OP, FailureAction.THROW_IO_EXCEPTION); } - private void testIOExceptionsDuringHandshake(boolean failRead, boolean failWrite) throws Exception { + /** + * Tests that if the remote end closes connection ungracefully during SSL handshake while reading data, + * the disconnection is not treated as an authentication failure. + */ + @Test + public void testUngracefulRemoteCloseDuringHandshakeRead() throws Exception { server = createEchoServer(SecurityProtocol.SSL); + testIOExceptionsDuringHandshake(server::closeSocketChannels, FailureAction.NO_OP); + } + + /** + * Tests that if the remote end closes connection ungracefully during SSL handshake while writing data, + * the disconnection is not treated as an authentication failure. + */ + @Test + public void testUngracefulRemoteCloseDuringHandshakeWrite() throws Exception { + server = createEchoServer(SecurityProtocol.SSL); + testIOExceptionsDuringHandshake(FailureAction.NO_OP, server::closeSocketChannels); + } + + /** + * Tests that if the remote end closes the connection during SSL handshake while reading data, + * the disconnection is not treated as an authentication failure. + */ + @Test + public void testGracefulRemoteCloseDuringHandshakeRead() throws Exception { + server = createEchoServer(SecurityProtocol.SSL); + testIOExceptionsDuringHandshake(FailureAction.NO_OP, server::closeKafkaChannels); + } + + /** + * Tests that if the remote end closes the connection during SSL handshake while writing data, + * the disconnection is not treated as an authentication failure. + */ + @Test + public void testGracefulRemoteCloseDuringHandshakeWrite() throws Exception { + server = createEchoServer(SecurityProtocol.SSL); + testIOExceptionsDuringHandshake(server::closeKafkaChannels, FailureAction.NO_OP); + } + + private void testIOExceptionsDuringHandshake(FailureAction readFailureAction, + FailureAction flushFailureAction) throws Exception { TestSslChannelBuilder channelBuilder = new TestSslChannelBuilder(Mode.CLIENT); boolean done = false; for (int i = 1; i <= 100; i++) { - int readFailureIndex = failRead ? i : Integer.MAX_VALUE; - int flushFailureIndex = failWrite ? i : Integer.MAX_VALUE; String node = String.valueOf(i); - channelBuilder.readFailureIndex = readFailureIndex; - channelBuilder.flushFailureIndex = flushFailureIndex; + channelBuilder.readFailureAction = readFailureAction; + channelBuilder.flushFailureAction = flushFailureAction; + channelBuilder.failureIndex = i; channelBuilder.configure(sslClientConfigs); this.selector = new Selector(5000, new Metrics(), new MockTime(), "MetricGroup", channelBuilder, new LogContext()); @@ -734,7 +796,9 @@ private void testIOExceptionsDuringHandshake(boolean failRead, boolean failWrite break; } if (selector.disconnected().containsKey(node)) { - assertEquals(ChannelState.State.AUTHENTICATE, selector.disconnected().get(node).state()); + ChannelState.State state = selector.disconnected().get(node).state(); + assertTrue("Unexpected channel state " + state, + state == ChannelState.State.AUTHENTICATE || state == ChannelState.State.READY); break; } } @@ -973,13 +1037,23 @@ private NioEchoServer createEchoServer(SecurityProtocol securityProtocol) throws return createEchoServer(ListenerName.forSecurityProtocol(securityProtocol), securityProtocol); } + @FunctionalInterface + private interface FailureAction { + FailureAction NO_OP = () -> { }; + FailureAction THROW_IO_EXCEPTION = () -> { + throw new IOException("Test IO exception"); + }; + void run() throws IOException; + } + private static class TestSslChannelBuilder extends SslChannelBuilder { private Integer netReadBufSizeOverride; private Integer netWriteBufSizeOverride; private Integer appBufSizeOverride; - long readFailureIndex = Long.MAX_VALUE; - long flushFailureIndex = Long.MAX_VALUE; + private long failureIndex = Long.MAX_VALUE; + FailureAction readFailureAction = FailureAction.NO_OP; + FailureAction flushFailureAction = FailureAction.NO_OP; int flushDelayCount = 0; public TestSslChannelBuilder(Mode mode) { @@ -997,7 +1071,6 @@ protected SslTransportLayer buildTransportLayer(SslFactory sslFactory, String id SocketChannel socketChannel = (SocketChannel) key.channel(); SSLEngine sslEngine = sslFactory.createSslEngine(host, socketChannel.socket().getPort()); TestSslTransportLayer transportLayer = newTransportLayer(id, key, sslEngine); - transportLayer.startHandshake(); return transportLayer; } @@ -1029,8 +1102,8 @@ public TestSslTransportLayer(String channelId, SelectionKey key, SSLEngine sslEn this.netReadBufSize = new ResizeableBufferSize(netReadBufSizeOverride); this.netWriteBufSize = new ResizeableBufferSize(netWriteBufSizeOverride); this.appBufSize = new ResizeableBufferSize(appBufSizeOverride); - numReadsRemaining = new AtomicLong(readFailureIndex); - numFlushesRemaining = new AtomicLong(flushFailureIndex); + numReadsRemaining = new AtomicLong(failureIndex); + numFlushesRemaining = new AtomicLong(failureIndex); numDelayedFlushesRemaining = new AtomicInteger(flushDelayCount); } @@ -1058,20 +1131,26 @@ protected int applicationBufferSize() { @Override protected int readFromSocketChannel() throws IOException { if (numReadsRemaining.decrementAndGet() == 0 && !ready()) - throw new IOException("Test exception during read"); + readFailureAction.run(); return super.readFromSocketChannel(); } @Override protected boolean flush(ByteBuffer buf) throws IOException { if (numFlushesRemaining.decrementAndGet() == 0 && !ready()) - throw new IOException("Test exception during write"); + flushFailureAction.run(); else if (numDelayedFlushesRemaining.getAndDecrement() != 0) return false; resetDelayedFlush(); return super.flush(buf); } + @Override + protected void startHandshake() throws IOException { + assertTrue("SSL handshake initialized too early", socketChannel().isConnected()); + super.startHandshake(); + } + private void resetDelayedFlush() { numDelayedFlushesRemaining.set(flushDelayCount); } diff --git a/clients/src/test/java/org/apache/kafka/common/record/BufferSupplierTest.java b/clients/src/test/java/org/apache/kafka/common/record/BufferSupplierTest.java new file mode 100644 index 0000000000000..dea0c9854133e --- /dev/null +++ b/clients/src/test/java/org/apache/kafka/common/record/BufferSupplierTest.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.record; + +import org.junit.Test; + +import java.nio.ByteBuffer; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertSame; + +public class BufferSupplierTest { + + @Test + public void testGrowableBuffer() { + BufferSupplier.GrowableBufferSupplier supplier = new BufferSupplier.GrowableBufferSupplier(); + ByteBuffer buffer = supplier.get(1024); + assertEquals(0, buffer.position()); + assertEquals(1024, buffer.capacity()); + supplier.release(buffer); + + ByteBuffer cached = supplier.get(512); + assertEquals(0, cached.position()); + assertSame(buffer, cached); + + ByteBuffer increased = supplier.get(2048); + assertEquals(2048, increased.capacity()); + assertEquals(0, increased.position()); + } + +} diff --git a/clients/src/test/java/org/apache/kafka/common/record/FileRecordsTest.java b/clients/src/test/java/org/apache/kafka/common/record/FileRecordsTest.java index f8b6dd4140c56..d43915559a42a 100644 --- a/clients/src/test/java/org/apache/kafka/common/record/FileRecordsTest.java +++ b/clients/src/test/java/org/apache/kafka/common/record/FileRecordsTest.java @@ -22,7 +22,6 @@ import org.apache.kafka.common.header.internals.RecordHeader; 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.easymock.EasyMock; import org.junit.Before; @@ -36,8 +35,10 @@ import java.util.Collections; import java.util.Iterator; import java.util.List; +import java.util.Random; import static java.util.Arrays.asList; +import static org.apache.kafka.common.utils.Utils.utf8; import static org.apache.kafka.test.TestUtils.tempFile; import static org.junit.Assert.assertArrayEquals; import static org.junit.Assert.assertEquals; @@ -121,7 +122,7 @@ public void testIterationDoesntChangePosition() throws IOException { */ @Test public void testRead() throws IOException { - FileRecords read = fileRecords.read(0, fileRecords.sizeInBytes()); + FileRecords read = fileRecords.slice(0, fileRecords.sizeInBytes()); assertEquals(fileRecords.sizeInBytes(), read.sizeInBytes()); TestUtils.checkEquals(fileRecords.batches(), read.batches()); @@ -129,35 +130,35 @@ public void testRead() throws IOException { RecordBatch first = items.get(0); // read from second message until the end - read = fileRecords.read(first.sizeInBytes(), fileRecords.sizeInBytes() - first.sizeInBytes()); + read = fileRecords.slice(first.sizeInBytes(), fileRecords.sizeInBytes() - first.sizeInBytes()); assertEquals(fileRecords.sizeInBytes() - first.sizeInBytes(), read.sizeInBytes()); assertEquals("Read starting from the second message", items.subList(1, items.size()), batches(read)); // read from second message and size is past the end of the file - read = fileRecords.read(first.sizeInBytes(), fileRecords.sizeInBytes()); + read = fileRecords.slice(first.sizeInBytes(), fileRecords.sizeInBytes()); assertEquals(fileRecords.sizeInBytes() - first.sizeInBytes(), read.sizeInBytes()); assertEquals("Read starting from the second message", items.subList(1, items.size()), batches(read)); // read from second message and position + size overflows - read = fileRecords.read(first.sizeInBytes(), Integer.MAX_VALUE); + read = fileRecords.slice(first.sizeInBytes(), Integer.MAX_VALUE); assertEquals(fileRecords.sizeInBytes() - first.sizeInBytes(), read.sizeInBytes()); assertEquals("Read starting from the second message", items.subList(1, items.size()), batches(read)); // read from second message and size is past the end of the file on a view/slice - read = fileRecords.read(1, fileRecords.sizeInBytes() - 1) - .read(first.sizeInBytes() - 1, fileRecords.sizeInBytes()); + read = fileRecords.slice(1, fileRecords.sizeInBytes() - 1) + .slice(first.sizeInBytes() - 1, fileRecords.sizeInBytes()); assertEquals(fileRecords.sizeInBytes() - first.sizeInBytes(), read.sizeInBytes()); assertEquals("Read starting from the second message", items.subList(1, items.size()), batches(read)); // read from second message and position + size overflows on a view/slice - read = fileRecords.read(1, fileRecords.sizeInBytes() - 1) - .read(first.sizeInBytes() - 1, Integer.MAX_VALUE); + read = fileRecords.slice(1, fileRecords.sizeInBytes() - 1) + .slice(first.sizeInBytes() - 1, Integer.MAX_VALUE); assertEquals(fileRecords.sizeInBytes() - first.sizeInBytes(), read.sizeInBytes()); assertEquals("Read starting from the second message", items.subList(1, items.size()), batches(read)); // read a single message starting from second message RecordBatch second = items.get(1); - read = fileRecords.read(first.sizeInBytes(), second.sizeInBytes()); + read = fileRecords.slice(first.sizeInBytes(), second.sizeInBytes()); assertEquals(second.sizeInBytes(), read.sizeInBytes()); assertEquals("Read a single message starting from the second message", Collections.singletonList(second), batches(read)); @@ -207,9 +208,9 @@ public void testIteratorWithLimits() throws IOException { RecordBatch batch = batches(fileRecords).get(1); int start = fileRecords.searchForOffsetWithSize(1, 0).position; int size = batch.sizeInBytes(); - FileRecords slice = fileRecords.read(start, size); + FileRecords slice = fileRecords.slice(start, size); assertEquals(Collections.singletonList(batch), batches(slice)); - FileRecords slice2 = fileRecords.read(start, size - 1); + FileRecords slice2 = fileRecords.slice(start, size - 1); assertEquals(Collections.emptyList(), batches(slice2)); } @@ -344,7 +345,7 @@ public void testFormatConversionWithPartialMessage() throws IOException { RecordBatch batch = batches(fileRecords).get(1); int start = fileRecords.searchForOffsetWithSize(1, 0).position; int size = batch.sizeInBytes(); - FileRecords slice = fileRecords.read(start, size - 1); + FileRecords slice = fileRecords.slice(start, size - 1); Records messageV0 = slice.downConvert(RecordBatch.MAGIC_VALUE_V0, 0, time).records(); assertTrue("No message should be there", batches(messageV0).isEmpty()); assertEquals("There should be " + (size - 1) + " bytes", size - 1, messageV0.sizeInBytes()); @@ -356,6 +357,39 @@ public void testFormatConversionWithPartialMessage() throws IOException { assertTrue("No messages should be returned", !it.hasNext()); } + @Test + public void testDownconversionAfterMessageFormatDowngrade() throws IOException { + // random bytes + Random random = new Random(); + byte[] bytes = new byte[3000]; + random.nextBytes(bytes); + + // records + CompressionType compressionType = CompressionType.GZIP; + List offsets = asList(0L, 1L); + List magic = asList(RecordBatch.MAGIC_VALUE_V2, RecordBatch.MAGIC_VALUE_V1); // downgrade message format from v2 to v1 + List records = asList( + new SimpleRecord(1L, "k1".getBytes(), bytes), + new SimpleRecord(2L, "k2".getBytes(), bytes)); + byte toMagic = 1; + + // create MemoryRecords + ByteBuffer buffer = ByteBuffer.allocate(8000); + for (int i = 0; i < records.size(); i++) { + MemoryRecordsBuilder builder = MemoryRecords.builder(buffer, magic.get(i), compressionType, TimestampType.CREATE_TIME, 0L); + builder.appendWithOffset(offsets.get(i), records.get(i)); + builder.close(); + } + buffer.flip(); + + // create FileRecords, down-convert and verify + try (FileRecords fileRecords = FileRecords.open(tempFile())) { + fileRecords.append(MemoryRecords.readableRecords(buffer)); + fileRecords.flush(); + downConvertAndVerifyRecords(records, offsets, fileRecords, compressionType, toMagic, 0L, time); + } + } + @Test public void testConversion() throws IOException { doTestConversion(CompressionType.NONE, RecordBatch.MAGIC_VALUE_V0); @@ -430,10 +464,6 @@ private void doTestConversion(CompressionType compressionType, byte toMagic) thr } } - private String utf8(ByteBuffer buffer) { - return Utils.utf8(buffer, buffer.remaining()); - } - private void downConvertAndVerifyRecords(List initialRecords, List initialOffsets, FileRecords fileRecords, @@ -441,13 +471,11 @@ private void downConvertAndVerifyRecords(List initialRecords, byte toMagic, long firstOffset, Time time) { - long numBatches = 0; long minBatchSize = Long.MAX_VALUE; long maxBatchSize = Long.MIN_VALUE; for (RecordBatch batch : fileRecords.batches()) { minBatchSize = Math.min(minBatchSize, batch.sizeInBytes()); maxBatchSize = Math.max(maxBatchSize, batch.sizeInBytes()); - numBatches++; } // Test the normal down-conversion path @@ -469,21 +497,6 @@ private void downConvertAndVerifyRecords(List initialRecords, Iterator it = lazyRecords.iterator(readSize); while (it.hasNext()) convertedRecords.add(it.next().records()); - - // Check if chunking works as expected. The only way to predictably test for this is by testing the edge cases. - // 1. If maximum read size is greater than the size of all batches combined, we must get all down-conversion - // records in exactly two batches; the first chunk is pre down-converted and returned, and the second chunk - // contains the remaining batches. - // 2. If maximum read size is just smaller than the size of all batches combined, we must get results in two - // chunks. - // 3. If maximum read size is less than the size of a single record, we get one batch in each chunk. - if (readSize >= fileRecords.sizeInBytes()) - assertEquals(2, convertedRecords.size()); - else if (readSize == fileRecords.sizeInBytes() - 1) - assertEquals(2, convertedRecords.size()); - else if (readSize <= minBatchSize) - assertEquals(numBatches, convertedRecords.size()); - verifyConvertedRecords(initialRecords, initialOffsets, convertedRecords, compressionType, toMagic); convertedRecords.clear(); } diff --git a/clients/src/test/java/org/apache/kafka/common/record/LazyDownConversionRecordsTest.java b/clients/src/test/java/org/apache/kafka/common/record/LazyDownConversionRecordsTest.java index 87656038f9bef..89c1aeac4c2f9 100644 --- a/clients/src/test/java/org/apache/kafka/common/record/LazyDownConversionRecordsTest.java +++ b/clients/src/test/java/org/apache/kafka/common/record/LazyDownConversionRecordsTest.java @@ -20,7 +20,6 @@ import org.apache.kafka.common.header.Header; import org.apache.kafka.common.header.internals.RecordHeader; import org.apache.kafka.common.utils.Time; -import org.apache.kafka.common.utils.Utils; import org.junit.Test; import org.junit.runner.RunWith; import org.junit.runners.Parameterized; @@ -35,86 +34,126 @@ import java.util.List; import static java.util.Arrays.asList; +import static org.apache.kafka.common.utils.Utils.utf8; import static org.apache.kafka.test.TestUtils.tempFile; import static org.junit.Assert.assertArrayEquals; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertFalse; import static org.junit.Assert.assertTrue; -@RunWith(value = Parameterized.class) public class LazyDownConversionRecordsTest { - private final CompressionType compressionType; - private final byte toMagic; - private final DownConversionTest test; - - public LazyDownConversionRecordsTest(CompressionType compressionType, byte toMagic, DownConversionTest test) { - this.compressionType = compressionType; - this.toMagic = toMagic; - this.test = test; + /** + * Test the lazy down-conversion path in the presence of commit markers. When converting to V0 or V1, these batches + * are dropped. If there happen to be no more batches left to convert, we must get an overflow message batch after + * conversion. + */ + @Test + public void testConversionOfCommitMarker() throws IOException { + MemoryRecords recordsToConvert = MemoryRecords.withEndTransactionMarker(0, Time.SYSTEM.milliseconds(), RecordBatch.NO_PARTITION_LEADER_EPOCH, + 1, (short) 1, new EndTransactionMarker(ControlRecordType.COMMIT, 0)); + MemoryRecords convertedRecords = convertRecords(recordsToConvert, (byte) 1, recordsToConvert.sizeInBytes()); + ByteBuffer buffer = convertedRecords.buffer(); + + // read the offset and the batch length + buffer.getLong(); + int sizeOfConvertedRecords = buffer.getInt(); + + // assert we got an overflow message batch + assertTrue(sizeOfConvertedRecords > buffer.limit()); + assertFalse(convertedRecords.batchIterator().hasNext()); } - enum DownConversionTest { - DEFAULT, - OVERFLOW, - } + @RunWith(value = Parameterized.class) + public static class ParameterizedConversionTest { + private final CompressionType compressionType; + private final byte toMagic; - @Parameterized.Parameters(name = "compressionType={0}, toMagic={1}, test={2}") - public static Collection data() { - List values = new ArrayList<>(); - for (byte toMagic = RecordBatch.MAGIC_VALUE_V0; toMagic <= RecordBatch.CURRENT_MAGIC_VALUE; toMagic++) { - for (DownConversionTest test : DownConversionTest.values()) { - values.add(new Object[]{CompressionType.NONE, toMagic, test}); - values.add(new Object[]{CompressionType.GZIP, toMagic, test}); + public ParameterizedConversionTest(CompressionType compressionType, byte toMagic) { + this.compressionType = compressionType; + this.toMagic = toMagic; + } + + @Parameterized.Parameters(name = "compressionType={0}, toMagic={1}") + public static Collection data() { + List values = new ArrayList<>(); + for (byte toMagic = RecordBatch.MAGIC_VALUE_V0; toMagic <= RecordBatch.CURRENT_MAGIC_VALUE; toMagic++) { + values.add(new Object[]{CompressionType.NONE, toMagic}); + values.add(new Object[]{CompressionType.GZIP, toMagic}); } + return values; } - return values; - } - @Test - public void doTestConversion() throws IOException { - List offsets = asList(0L, 2L, 3L, 9L, 11L, 15L, 16L, 17L, 22L, 24L); - - Header[] headers = {new RecordHeader("headerKey1", "headerValue1".getBytes()), - new RecordHeader("headerKey2", "headerValue2".getBytes()), - new RecordHeader("headerKey3", "headerValue3".getBytes())}; - - List records = asList( - new SimpleRecord(1L, "k1".getBytes(), "hello".getBytes()), - new SimpleRecord(2L, "k2".getBytes(), "goodbye".getBytes()), - new SimpleRecord(3L, "k3".getBytes(), "hello again".getBytes()), - new SimpleRecord(4L, "k4".getBytes(), "goodbye for now".getBytes()), - new SimpleRecord(5L, "k5".getBytes(), "hello again".getBytes()), - new SimpleRecord(6L, "k6".getBytes(), "I sense indecision".getBytes()), - new SimpleRecord(7L, "k7".getBytes(), "what now".getBytes()), - new SimpleRecord(8L, "k8".getBytes(), "running out".getBytes(), headers), - new SimpleRecord(9L, "k9".getBytes(), "ok, almost done".getBytes()), - new SimpleRecord(10L, "k10".getBytes(), "finally".getBytes(), headers)); - assertEquals("incorrect test setup", offsets.size(), records.size()); - - ByteBuffer buffer = ByteBuffer.allocate(1024); - MemoryRecordsBuilder builder = MemoryRecords.builder(buffer, RecordBatch.CURRENT_MAGIC_VALUE, compressionType, - TimestampType.CREATE_TIME, 0L); - for (int i = 0; i < 3; i++) - builder.appendWithOffset(offsets.get(i), records.get(i)); - builder.close(); - - builder = MemoryRecords.builder(buffer, RecordBatch.CURRENT_MAGIC_VALUE, compressionType, TimestampType.CREATE_TIME, - 0L); - for (int i = 3; i < 6; i++) - builder.appendWithOffset(offsets.get(i), records.get(i)); - builder.close(); - - builder = MemoryRecords.builder(buffer, RecordBatch.CURRENT_MAGIC_VALUE, compressionType, TimestampType.CREATE_TIME, - 0L); - for (int i = 6; i < 10; i++) - builder.appendWithOffset(offsets.get(i), records.get(i)); - builder.close(); - - buffer.flip(); + /** + * Test the lazy down-conversion path. + */ + @Test + public void testConversion() throws IOException { + doTestConversion(false); + } + + /** + * Test the lazy down-conversion path where the number of bytes we want to convert is much larger than the + * number of bytes we get after conversion. This causes overflow message batch(es) to be appended towards the + * end of the converted output. + */ + @Test + public void testConversionWithOverflow() throws IOException { + doTestConversion(true); + } + private void doTestConversion(boolean testConversionOverflow) throws IOException { + List offsets = asList(0L, 2L, 3L, 9L, 11L, 15L, 16L, 17L, 22L, 24L); + + Header[] headers = {new RecordHeader("headerKey1", "headerValue1".getBytes()), + new RecordHeader("headerKey2", "headerValue2".getBytes()), + new RecordHeader("headerKey3", "headerValue3".getBytes())}; + + List records = asList( + new SimpleRecord(1L, "k1".getBytes(), "hello".getBytes()), + new SimpleRecord(2L, "k2".getBytes(), "goodbye".getBytes()), + new SimpleRecord(3L, "k3".getBytes(), "hello again".getBytes()), + new SimpleRecord(4L, "k4".getBytes(), "goodbye for now".getBytes()), + new SimpleRecord(5L, "k5".getBytes(), "hello again".getBytes()), + new SimpleRecord(6L, "k6".getBytes(), "I sense indecision".getBytes()), + new SimpleRecord(7L, "k7".getBytes(), "what now".getBytes()), + new SimpleRecord(8L, "k8".getBytes(), "running out".getBytes(), headers), + new SimpleRecord(9L, "k9".getBytes(), "ok, almost done".getBytes()), + new SimpleRecord(10L, "k10".getBytes(), "finally".getBytes(), headers)); + assertEquals("incorrect test setup", offsets.size(), records.size()); + + ByteBuffer buffer = ByteBuffer.allocate(1024); + MemoryRecordsBuilder builder = MemoryRecords.builder(buffer, RecordBatch.CURRENT_MAGIC_VALUE, compressionType, + TimestampType.CREATE_TIME, 0L); + for (int i = 0; i < 3; i++) + builder.appendWithOffset(offsets.get(i), records.get(i)); + builder.close(); + + builder = MemoryRecords.builder(buffer, RecordBatch.CURRENT_MAGIC_VALUE, compressionType, TimestampType.CREATE_TIME, + 0L); + for (int i = 3; i < 6; i++) + builder.appendWithOffset(offsets.get(i), records.get(i)); + builder.close(); + + builder = MemoryRecords.builder(buffer, RecordBatch.CURRENT_MAGIC_VALUE, compressionType, TimestampType.CREATE_TIME, + 0L); + for (int i = 6; i < 10; i++) + builder.appendWithOffset(offsets.get(i), records.get(i)); + builder.close(); + buffer.flip(); + + MemoryRecords recordsToConvert = MemoryRecords.readableRecords(buffer); + int numBytesToConvert = recordsToConvert.sizeInBytes(); + if (testConversionOverflow) + numBytesToConvert *= 2; + + MemoryRecords convertedRecords = convertRecords(recordsToConvert, toMagic, numBytesToConvert); + verifyDownConvertedRecords(records, offsets, convertedRecords, compressionType, toMagic); + } + } + + private static MemoryRecords convertRecords(MemoryRecords recordsToConvert, byte toMagic, int bytesToConvert) throws IOException { try (FileRecords inputRecords = FileRecords.open(tempFile())) { - MemoryRecords memoryRecords = MemoryRecords.readableRecords(buffer); - inputRecords.append(memoryRecords); + inputRecords.append(recordsToConvert); inputRecords.flush(); LazyDownConversionRecords lazyRecords = new LazyDownConversionRecords(new TopicPartition("test", 1), @@ -123,50 +162,27 @@ public void doTestConversion() throws IOException { File outputFile = tempFile(); FileChannel channel = new RandomAccessFile(outputFile, "rw").getChannel(); - // Size of lazy records is at least as much as the size of underlying records - assertTrue(lazyRecords.sizeInBytes() >= inputRecords.sizeInBytes()); - - int toWrite; int written = 0; - List recordsBeingConverted; - List offsetsOfRecords; - switch (test) { - case DEFAULT: - toWrite = inputRecords.sizeInBytes(); - recordsBeingConverted = records; - offsetsOfRecords = offsets; - break; - case OVERFLOW: - toWrite = inputRecords.sizeInBytes() * 2; - recordsBeingConverted = records; - offsetsOfRecords = offsets; - break; - default: - throw new IllegalArgumentException(); - } - while (written < toWrite) - written += lazySend.writeTo(channel, written, toWrite - written); + while (written < bytesToConvert) + written += lazySend.writeTo(channel, written, bytesToConvert - written); FileRecords convertedRecords = FileRecords.open(outputFile, true, (int) channel.size(), false); ByteBuffer convertedRecordsBuffer = ByteBuffer.allocate(convertedRecords.sizeInBytes()); convertedRecords.readInto(convertedRecordsBuffer, 0); - MemoryRecords convertedMemoryRecords = MemoryRecords.readableRecords(convertedRecordsBuffer); - verifyDownConvertedRecords(recordsBeingConverted, offsetsOfRecords, convertedMemoryRecords, compressionType, toMagic); + // cleanup convertedRecords.close(); channel.close(); - } - } - private String utf8(ByteBuffer buffer) { - return Utils.utf8(buffer, buffer.remaining()); + return MemoryRecords.readableRecords(convertedRecordsBuffer); + } } - private void verifyDownConvertedRecords(List initialRecords, - List initialOffsets, - MemoryRecords downConvertedRecords, - CompressionType compressionType, - byte toMagic) { + private static void verifyDownConvertedRecords(List initialRecords, + List initialOffsets, + MemoryRecords downConvertedRecords, + CompressionType compressionType, + byte toMagic) { int i = 0; for (RecordBatch batch : downConvertedRecords.batches()) { assertTrue("Magic byte should be lower than or equal to " + toMagic, batch.magic() <= toMagic); diff --git a/clients/src/test/java/org/apache/kafka/common/record/MemoryRecordsBuilderTest.java b/clients/src/test/java/org/apache/kafka/common/record/MemoryRecordsBuilderTest.java index 36b14a2f40dcb..5d5221ecceac9 100644 --- a/clients/src/test/java/org/apache/kafka/common/record/MemoryRecordsBuilderTest.java +++ b/clients/src/test/java/org/apache/kafka/common/record/MemoryRecordsBuilderTest.java @@ -30,6 +30,7 @@ import java.util.List; import java.util.Random; +import static org.apache.kafka.common.utils.Utils.utf8; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertFalse; import static org.junit.Assert.assertNotNull; @@ -568,10 +569,6 @@ public void convertToV1WithMixedV0AndV2Data() { } } - private String utf8(ByteBuffer buffer) { - return Utils.utf8(buffer, buffer.remaining()); - } - @Test public void shouldThrowIllegalStateExceptionOnBuildWhenAborted() throws Exception { ByteBuffer buffer = ByteBuffer.allocate(128); 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 61d8a00865bc5..579fb74b44a83 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 @@ -252,6 +252,7 @@ public void testFilterToEmptyBatchRetention() { long baseOffset = 3L; int baseSequence = 10; int partitionLeaderEpoch = 293; + int numRecords = 2; MemoryRecordsBuilder builder = MemoryRecords.builder(buffer, magic, compression, TimestampType.CREATE_TIME, baseOffset, RecordBatch.NO_TIMESTAMP, producerId, producerEpoch, baseSequence, isTransactional, @@ -259,22 +260,34 @@ public void testFilterToEmptyBatchRetention() { builder.append(11L, "2".getBytes(), "b".getBytes()); builder.append(12L, "3".getBytes(), "c".getBytes()); builder.close(); + MemoryRecords records = builder.build(); ByteBuffer filtered = ByteBuffer.allocate(2048); - builder.build().filterTo(new TopicPartition("foo", 0), new MemoryRecords.RecordFilter() { - @Override - protected BatchRetention checkBatchRetention(RecordBatch batch) { - // retain all batches - return BatchRetention.RETAIN_EMPTY; - } - - @Override - protected boolean shouldRetainRecord(RecordBatch recordBatch, Record record) { - // delete the records - return false; - } - }, filtered, Integer.MAX_VALUE, BufferSupplier.NO_CACHING); - + MemoryRecords.FilterResult filterResult = records.filterTo(new TopicPartition("foo", 0), + new MemoryRecords.RecordFilter() { + @Override + protected BatchRetention checkBatchRetention(RecordBatch batch) { + // retain all batches + return BatchRetention.RETAIN_EMPTY; + } + + @Override + protected boolean shouldRetainRecord(RecordBatch recordBatch, Record record) { + // delete the records + return false; + } + }, filtered, Integer.MAX_VALUE, BufferSupplier.NO_CACHING); + + // Verify filter result + assertEquals(numRecords, filterResult.messagesRead()); + assertEquals(records.sizeInBytes(), filterResult.bytesRead()); + assertEquals(baseOffset + 1, filterResult.maxOffset()); + assertEquals(0, filterResult.messagesRetained()); + assertEquals(DefaultRecordBatch.RECORD_BATCH_OVERHEAD, filterResult.bytesRetained()); + assertEquals(12, filterResult.maxTimestamp()); + assertEquals(baseOffset + 1, filterResult.shallowOffsetOfMaxTimestamp()); + + // Verify filtered records filtered.flip(); MemoryRecords filteredRecords = MemoryRecords.readableRecords(filtered); @@ -294,6 +307,55 @@ protected boolean shouldRetainRecord(RecordBatch recordBatch, Record record) { } } + @Test + public void testEmptyBatchRetention() { + if (magic >= RecordBatch.MAGIC_VALUE_V2) { + ByteBuffer buffer = ByteBuffer.allocate(DefaultRecordBatch.RECORD_BATCH_OVERHEAD); + long producerId = 23L; + short producerEpoch = 5; + long baseOffset = 3L; + int baseSequence = 10; + int partitionLeaderEpoch = 293; + long timestamp = System.currentTimeMillis(); + + DefaultRecordBatch.writeEmptyHeader(buffer, RecordBatch.MAGIC_VALUE_V2, producerId, producerEpoch, + baseSequence, baseOffset, baseOffset, partitionLeaderEpoch, TimestampType.CREATE_TIME, + timestamp, false, false); + buffer.flip(); + + ByteBuffer filtered = ByteBuffer.allocate(2048); + MemoryRecords records = MemoryRecords.readableRecords(buffer); + MemoryRecords.FilterResult filterResult = records.filterTo(new TopicPartition("foo", 0), + new MemoryRecords.RecordFilter() { + @Override + protected BatchRetention checkBatchRetention(RecordBatch batch) { + // retain all batches + return BatchRetention.RETAIN_EMPTY; + } + + @Override + protected boolean shouldRetainRecord(RecordBatch recordBatch, Record record) { + return false; + } + }, filtered, Integer.MAX_VALUE, BufferSupplier.NO_CACHING); + + // Verify filter result + assertEquals(0, filterResult.messagesRead()); + assertEquals(records.sizeInBytes(), filterResult.bytesRead()); + assertEquals(baseOffset, filterResult.maxOffset()); + assertEquals(0, filterResult.messagesRetained()); + assertEquals(DefaultRecordBatch.RECORD_BATCH_OVERHEAD, filterResult.bytesRetained()); + assertEquals(timestamp, filterResult.maxTimestamp()); + assertEquals(baseOffset, filterResult.shallowOffsetOfMaxTimestamp()); + assertTrue(filterResult.outputBuffer().position() > 0); + + // Verify filtered records + filtered.flip(); + MemoryRecords filteredRecords = MemoryRecords.readableRecords(filtered); + assertEquals(DefaultRecordBatch.RECORD_BATCH_OVERHEAD, filteredRecords.sizeInBytes()); + } + } + @Test public void testEmptyBatchDeletion() { if (magic >= RecordBatch.MAGIC_VALUE_V2) { @@ -304,25 +366,32 @@ public void testEmptyBatchDeletion() { long baseOffset = 3L; int baseSequence = 10; int partitionLeaderEpoch = 293; + long timestamp = System.currentTimeMillis(); DefaultRecordBatch.writeEmptyHeader(buffer, RecordBatch.MAGIC_VALUE_V2, producerId, producerEpoch, baseSequence, baseOffset, baseOffset, partitionLeaderEpoch, TimestampType.CREATE_TIME, - System.currentTimeMillis(), false, false); + timestamp, false, false); buffer.flip(); ByteBuffer filtered = ByteBuffer.allocate(2048); - MemoryRecords.readableRecords(buffer).filterTo(new TopicPartition("foo", 0), new MemoryRecords.RecordFilter() { - @Override - protected BatchRetention checkBatchRetention(RecordBatch batch) { - return deleteRetention; - } - - @Override - protected boolean shouldRetainRecord(RecordBatch recordBatch, Record record) { - return false; - } - }, filtered, Integer.MAX_VALUE, BufferSupplier.NO_CACHING); - + MemoryRecords records = MemoryRecords.readableRecords(buffer); + MemoryRecords.FilterResult filterResult = records.filterTo(new TopicPartition("foo", 0), + new MemoryRecords.RecordFilter() { + @Override + protected BatchRetention checkBatchRetention(RecordBatch batch) { + return deleteRetention; + } + + @Override + protected boolean shouldRetainRecord(RecordBatch recordBatch, Record record) { + return false; + } + }, filtered, Integer.MAX_VALUE, BufferSupplier.NO_CACHING); + + // Verify filter result + assertEquals(0, filterResult.outputBuffer().position()); + + // Verify filtered records filtered.flip(); MemoryRecords filteredRecords = MemoryRecords.readableRecords(filtered); assertEquals(0, filteredRecords.sizeInBytes()); @@ -591,15 +660,15 @@ public void testFilterToWithUndersizedBuffer() { MemoryRecords.FilterResult result = MemoryRecords.readableRecords(buffer) .filterTo(new TopicPartition("foo", 0), new RetainNonNullKeysFilter(), output, Integer.MAX_VALUE, - BufferSupplier.NO_CACHING); + BufferSupplier.NO_CACHING); - buffer.position(buffer.position() + result.bytesRead); - result.output.flip(); + buffer.position(buffer.position() + result.bytesRead()); + result.outputBuffer().flip(); - if (output != result.output) + if (output != result.outputBuffer()) assertEquals(0, output.position()); - MemoryRecords filtered = MemoryRecords.readableRecords(result.output); + MemoryRecords filtered = MemoryRecords.readableRecords(result.outputBuffer()); records.addAll(TestUtils.toList(filtered.records())); } @@ -623,9 +692,9 @@ public void testToString() { break; case RecordBatch.MAGIC_VALUE_V1: assertEquals("[(record=LegacyRecordBatch(offset=0, Record(magic=1, attributes=0, compression=NONE, " + - "crc=97210616, CreateTime=1000000, key=4 bytes, value=6 bytes))), (record=LegacyRecordBatch(offset=1, " + - "Record(magic=1, attributes=0, compression=NONE, crc=3535988507, CreateTime=1000001, key=4 bytes, " + - "value=6 bytes)))]", + "crc=97210616, CreateTime=1000000, key=4 bytes, value=6 bytes))), (record=LegacyRecordBatch(offset=1, " + + "Record(magic=1, attributes=0, compression=NONE, crc=3535988507, CreateTime=1000001, key=4 bytes, " + + "value=6 bytes)))]", memoryRecords.toString()); break; case RecordBatch.MAGIC_VALUE_V2: @@ -669,16 +738,16 @@ public void testFilterTo() { filtered.flip(); - assertEquals(7, result.messagesRead); - assertEquals(4, result.messagesRetained); - assertEquals(buffer.limit(), result.bytesRead); - assertEquals(filtered.limit(), result.bytesRetained); + assertEquals(7, result.messagesRead()); + assertEquals(4, result.messagesRetained()); + assertEquals(buffer.limit(), result.bytesRead()); + assertEquals(filtered.limit(), result.bytesRetained()); if (magic > RecordBatch.MAGIC_VALUE_V0) { - assertEquals(20L, result.maxTimestamp); + assertEquals(20L, result.maxTimestamp()); if (compression == CompressionType.NONE && magic < RecordBatch.MAGIC_VALUE_V2) - assertEquals(4L, result.shallowOffsetOfMaxTimestamp); + assertEquals(4L, result.shallowOffsetOfMaxTimestamp()); else - assertEquals(5L, result.shallowOffsetOfMaxTimestamp); + assertEquals(5L, result.shallowOffsetOfMaxTimestamp()); } MemoryRecords filteredRecords = MemoryRecords.readableRecords(filtered); diff --git a/clients/src/test/java/org/apache/kafka/common/requests/CreateAclsRequestTest.java b/clients/src/test/java/org/apache/kafka/common/requests/CreateAclsRequestTest.java index 748914b493137..5642677b0e11c 100644 --- a/clients/src/test/java/org/apache/kafka/common/requests/CreateAclsRequestTest.java +++ b/clients/src/test/java/org/apache/kafka/common/requests/CreateAclsRequestTest.java @@ -24,8 +24,8 @@ import org.apache.kafka.common.errors.UnsupportedVersionException; import org.apache.kafka.common.protocol.types.Struct; import org.apache.kafka.common.requests.CreateAclsRequest.AclCreation; -import org.apache.kafka.common.resource.Resource; -import org.apache.kafka.common.resource.ResourceNameType; +import org.apache.kafka.common.resource.PatternType; +import org.apache.kafka.common.resource.ResourcePattern; import org.apache.kafka.common.resource.ResourceType; import org.junit.Test; @@ -39,13 +39,16 @@ public class CreateAclsRequestTest { private static final short V0 = 0; private static final short V1 = 1; - private static final AclBinding LITERAL_ACL1 = new AclBinding(new Resource(ResourceType.TOPIC, "foo", ResourceNameType.LITERAL), + private static final AclBinding LITERAL_ACL1 = new AclBinding(new ResourcePattern(ResourceType.TOPIC, "foo", PatternType.LITERAL), new AccessControlEntry("User:ANONYMOUS", "127.0.0.1", AclOperation.READ, AclPermissionType.DENY)); - private static final AclBinding LITERAL_ACL2 = new AclBinding(new Resource(ResourceType.GROUP, "group", ResourceNameType.LITERAL), + private static final AclBinding LITERAL_ACL2 = new AclBinding(new ResourcePattern(ResourceType.GROUP, "group", PatternType.LITERAL), new AccessControlEntry("User:*", "127.0.0.1", AclOperation.WRITE, AclPermissionType.ALLOW)); - private static final AclBinding PREFIXED_ACL1 = new AclBinding(new Resource(ResourceType.GROUP, "prefix", ResourceNameType.PREFIXED), + private static final AclBinding PREFIXED_ACL1 = new AclBinding(new ResourcePattern(ResourceType.GROUP, "prefix", PatternType.PREFIXED), + new AccessControlEntry("User:*", "127.0.0.1", AclOperation.CREATE, AclPermissionType.ALLOW)); + + private static final AclBinding UNKNOWN_ACL1 = new AclBinding(new ResourcePattern(ResourceType.UNKNOWN, "unknown", PatternType.LITERAL), new AccessControlEntry("User:*", "127.0.0.1", AclOperation.CREATE, AclPermissionType.ALLOW)); @Test(expected = UnsupportedVersionException.class) @@ -53,6 +56,11 @@ public void shouldThrowOnV0IfNotLiteral() { new CreateAclsRequest(V0, aclCreations(PREFIXED_ACL1)); } + @Test(expected = IllegalArgumentException.class) + public void shouldThrowOnIfUnknown() { + new CreateAclsRequest(V0, aclCreations(UNKNOWN_ACL1)); + } + @Test public void shouldRoundTripV0() { final CreateAclsRequest original = new CreateAclsRequest(V0, aclCreations(LITERAL_ACL1, LITERAL_ACL2)); diff --git a/clients/src/test/java/org/apache/kafka/common/requests/DeleteAclsRequestTest.java b/clients/src/test/java/org/apache/kafka/common/requests/DeleteAclsRequestTest.java index 7761337867148..9be8d59608015 100644 --- a/clients/src/test/java/org/apache/kafka/common/requests/DeleteAclsRequestTest.java +++ b/clients/src/test/java/org/apache/kafka/common/requests/DeleteAclsRequestTest.java @@ -23,8 +23,8 @@ import org.apache.kafka.common.acl.AclPermissionType; import org.apache.kafka.common.errors.UnsupportedVersionException; import org.apache.kafka.common.protocol.types.Struct; -import org.apache.kafka.common.resource.ResourceFilter; -import org.apache.kafka.common.resource.ResourceNameType; +import org.apache.kafka.common.resource.PatternType; +import org.apache.kafka.common.resource.ResourcePatternFilter; import org.apache.kafka.common.resource.ResourceType; import org.junit.Test; @@ -37,22 +37,30 @@ public class DeleteAclsRequestTest { private static final short V0 = 0; private static final short V1 = 1; - private static final AclBindingFilter LITERAL_FILTER = new AclBindingFilter(new ResourceFilter(ResourceType.TOPIC, "foo", ResourceNameType.LITERAL), + private static final AclBindingFilter LITERAL_FILTER = new AclBindingFilter(new ResourcePatternFilter(ResourceType.TOPIC, "foo", PatternType.LITERAL), new AccessControlEntryFilter("User:ANONYMOUS", "127.0.0.1", AclOperation.READ, AclPermissionType.DENY)); - private static final AclBindingFilter PREFIXED_FILTER = new AclBindingFilter(new ResourceFilter(ResourceType.GROUP, "prefix", ResourceNameType.PREFIXED), + private static final AclBindingFilter PREFIXED_FILTER = new AclBindingFilter(new ResourcePatternFilter(ResourceType.GROUP, "prefix", PatternType.PREFIXED), new AccessControlEntryFilter("User:*", "127.0.0.1", AclOperation.CREATE, AclPermissionType.ALLOW)); - private static final AclBindingFilter ANY_FILTER = new AclBindingFilter(new ResourceFilter(ResourceType.GROUP, "prefix", ResourceNameType.PREFIXED), + private static final AclBindingFilter ANY_FILTER = new AclBindingFilter(new ResourcePatternFilter(ResourceType.GROUP, "bar", PatternType.ANY), + new AccessControlEntryFilter("User:*", "127.0.0.1", AclOperation.CREATE, AclPermissionType.ALLOW)); + + private static final AclBindingFilter UNKNOWN_FILTER = new AclBindingFilter(new ResourcePatternFilter(ResourceType.UNKNOWN, "prefix", PatternType.PREFIXED), new AccessControlEntryFilter("User:*", "127.0.0.1", AclOperation.CREATE, AclPermissionType.ALLOW)); @Test(expected = UnsupportedVersionException.class) - public void shouldThrowOnV0IfNotLiteral() { + public void shouldThrowOnV0IfPrefixed() { new DeleteAclsRequest(V0, aclFilters(PREFIXED_FILTER)); } + @Test(expected = IllegalArgumentException.class) + public void shouldThrowOnUnknownElements() { + new DeleteAclsRequest(V1, aclFilters(UNKNOWN_FILTER)); + } + @Test - public void shouldRoundTripV0() { + public void shouldRoundTripLiteralV0() { final DeleteAclsRequest original = new DeleteAclsRequest(V0, aclFilters(LITERAL_FILTER)); final Struct struct = original.toStruct(); @@ -61,6 +69,21 @@ public void shouldRoundTripV0() { assertRequestEquals(original, result); } + @Test + public void shouldRoundTripAnyV0AsLiteral() { + final DeleteAclsRequest original = new DeleteAclsRequest(V0, aclFilters(ANY_FILTER)); + final DeleteAclsRequest expected = new DeleteAclsRequest(V0, aclFilters( + new AclBindingFilter(new ResourcePatternFilter( + ANY_FILTER.patternFilter().resourceType(), + ANY_FILTER.patternFilter().name(), + PatternType.LITERAL), + ANY_FILTER.entryFilter()))); + + final DeleteAclsRequest result = new DeleteAclsRequest(original.toStruct(), V0); + + assertRequestEquals(expected, result); + } + @Test public void shouldRoundTripV1() { final DeleteAclsRequest original = new DeleteAclsRequest(V1, aclFilters(LITERAL_FILTER, PREFIXED_FILTER, ANY_FILTER)); diff --git a/clients/src/test/java/org/apache/kafka/common/requests/DeleteAclsResponseTest.java b/clients/src/test/java/org/apache/kafka/common/requests/DeleteAclsResponseTest.java index f8e9148cbd1e6..f8bec15c00bb5 100644 --- a/clients/src/test/java/org/apache/kafka/common/requests/DeleteAclsResponseTest.java +++ b/clients/src/test/java/org/apache/kafka/common/requests/DeleteAclsResponseTest.java @@ -21,12 +21,12 @@ import org.apache.kafka.common.acl.AclBinding; import org.apache.kafka.common.acl.AclOperation; import org.apache.kafka.common.acl.AclPermissionType; +import org.apache.kafka.common.resource.PatternType; +import org.apache.kafka.common.resource.ResourcePattern; import org.apache.kafka.common.errors.UnsupportedVersionException; import org.apache.kafka.common.protocol.types.Struct; import org.apache.kafka.common.requests.DeleteAclsResponse.AclDeletionResult; import org.apache.kafka.common.requests.DeleteAclsResponse.AclFilterResponse; -import org.apache.kafka.common.resource.Resource; -import org.apache.kafka.common.resource.ResourceNameType; import org.apache.kafka.common.resource.ResourceType; import org.junit.Test; @@ -41,24 +41,34 @@ public class DeleteAclsResponseTest { private static final short V0 = 0; private static final short V1 = 1; - private static final AclBinding LITERAL_ACL1 = new AclBinding(new org.apache.kafka.common.resource.Resource(ResourceType.TOPIC, "foo", ResourceNameType.LITERAL), + private static final AclBinding LITERAL_ACL1 = new AclBinding(new ResourcePattern(ResourceType.TOPIC, "foo", PatternType.LITERAL), new AccessControlEntry("User:ANONYMOUS", "127.0.0.1", AclOperation.READ, AclPermissionType.DENY)); - private static final AclBinding LITERAL_ACL2 = new AclBinding(new org.apache.kafka.common.resource.Resource(ResourceType.GROUP, "group", ResourceNameType.LITERAL), + private static final AclBinding LITERAL_ACL2 = new AclBinding(new ResourcePattern(ResourceType.GROUP, "group", PatternType.LITERAL), new AccessControlEntry("User:*", "127.0.0.1", AclOperation.WRITE, AclPermissionType.ALLOW)); - private static final AclBinding PREFIXED_ACL1 = new AclBinding(new Resource(ResourceType.GROUP, "prefix", ResourceNameType.PREFIXED), + private static final AclBinding PREFIXED_ACL1 = new AclBinding(new ResourcePattern(ResourceType.GROUP, "prefix", PatternType.PREFIXED), new AccessControlEntry("User:*", "127.0.0.1", AclOperation.CREATE, AclPermissionType.ALLOW)); + private static final AclBinding UNKNOWN_ACL = new AclBinding(new ResourcePattern(ResourceType.UNKNOWN, "group", PatternType.LITERAL), + new AccessControlEntry("User:*", "127.0.0.1", AclOperation.WRITE, AclPermissionType.ALLOW)); + private static final AclFilterResponse LITERAL_RESPONSE = new AclFilterResponse(aclDeletions(LITERAL_ACL1, LITERAL_ACL2)); private static final AclFilterResponse PREFIXED_RESPONSE = new AclFilterResponse(aclDeletions(LITERAL_ACL1, PREFIXED_ACL1)); + private static final AclFilterResponse UNKNOWN_RESPONSE = new AclFilterResponse(aclDeletions(UNKNOWN_ACL)); + @Test(expected = UnsupportedVersionException.class) public void shouldThrowOnV0IfNotLiteral() { new DeleteAclsResponse(10, aclResponses(PREFIXED_RESPONSE)).toStruct(V0); } + @Test(expected = IllegalArgumentException.class) + public void shouldThrowOnIfUnknown() { + new DeleteAclsResponse(10, aclResponses(UNKNOWN_RESPONSE)).toStruct(V1); + } + @Test public void shouldRoundTripV0() { final DeleteAclsResponse original = new DeleteAclsResponse(10, aclResponses(LITERAL_RESPONSE)); diff --git a/clients/src/test/java/org/apache/kafka/common/requests/DescribeAclsRequestTest.java b/clients/src/test/java/org/apache/kafka/common/requests/DescribeAclsRequestTest.java index 543cf37576ca7..7d9d1b1416c1e 100644 --- a/clients/src/test/java/org/apache/kafka/common/requests/DescribeAclsRequestTest.java +++ b/clients/src/test/java/org/apache/kafka/common/requests/DescribeAclsRequestTest.java @@ -23,8 +23,8 @@ import org.apache.kafka.common.acl.AclPermissionType; import org.apache.kafka.common.errors.UnsupportedVersionException; import org.apache.kafka.common.protocol.types.Struct; -import org.apache.kafka.common.resource.ResourceFilter; -import org.apache.kafka.common.resource.ResourceNameType; +import org.apache.kafka.common.resource.PatternType; +import org.apache.kafka.common.resource.ResourcePatternFilter; import org.apache.kafka.common.resource.ResourceType; import org.junit.Test; @@ -34,22 +34,30 @@ public class DescribeAclsRequestTest { private static final short V0 = 0; private static final short V1 = 1; - private static final AclBindingFilter LITERAL_FILTER = new AclBindingFilter(new ResourceFilter(ResourceType.TOPIC, "foo", ResourceNameType.LITERAL), + private static final AclBindingFilter LITERAL_FILTER = new AclBindingFilter(new ResourcePatternFilter(ResourceType.TOPIC, "foo", PatternType.LITERAL), new AccessControlEntryFilter("User:ANONYMOUS", "127.0.0.1", AclOperation.READ, AclPermissionType.DENY)); - private static final AclBindingFilter PREFIXED_FILTER = new AclBindingFilter(new ResourceFilter(ResourceType.GROUP, "prefix", ResourceNameType.PREFIXED), + private static final AclBindingFilter PREFIXED_FILTER = new AclBindingFilter(new ResourcePatternFilter(ResourceType.GROUP, "prefix", PatternType.PREFIXED), new AccessControlEntryFilter("User:*", "127.0.0.1", AclOperation.CREATE, AclPermissionType.ALLOW)); - private static final AclBindingFilter ANY_FILTER = new AclBindingFilter(new ResourceFilter(ResourceType.GROUP, "prefix", ResourceNameType.PREFIXED), + private static final AclBindingFilter ANY_FILTER = new AclBindingFilter(new ResourcePatternFilter(ResourceType.GROUP, "bar", PatternType.ANY), new AccessControlEntryFilter("User:*", "127.0.0.1", AclOperation.CREATE, AclPermissionType.ALLOW)); + private static final AclBindingFilter UNKNOWN_FILTER = new AclBindingFilter(new ResourcePatternFilter(ResourceType.UNKNOWN, "foo", PatternType.LITERAL), + new AccessControlEntryFilter("User:ANONYMOUS", "127.0.0.1", AclOperation.READ, AclPermissionType.DENY)); + @Test(expected = UnsupportedVersionException.class) - public void shouldThrowOnV0IfNotLiteral() { + public void shouldThrowOnV0IfPrefixed() { new DescribeAclsRequest(PREFIXED_FILTER, V0); } + @Test(expected = IllegalArgumentException.class) + public void shouldThrowIfUnknown() { + new DescribeAclsRequest(UNKNOWN_FILTER, V0); + } + @Test - public void shouldRoundTripV0() { + public void shouldRoundTripLiteralV0() { final DescribeAclsRequest original = new DescribeAclsRequest(LITERAL_FILTER, V0); final Struct struct = original.toStruct(); @@ -58,6 +66,22 @@ public void shouldRoundTripV0() { assertRequestEquals(original, result); } + @Test + public void shouldRoundTripAnyV0AsLiteral() { + final DescribeAclsRequest original = new DescribeAclsRequest(ANY_FILTER, V0); + final DescribeAclsRequest expected = new DescribeAclsRequest( + new AclBindingFilter(new ResourcePatternFilter( + ANY_FILTER.patternFilter().resourceType(), + ANY_FILTER.patternFilter().name(), + PatternType.LITERAL), + ANY_FILTER.entryFilter()), V0); + + final Struct struct = original.toStruct(); + final DescribeAclsRequest result = new DescribeAclsRequest(struct, V0); + + assertRequestEquals(expected, result); + } + @Test public void shouldRoundTripLiteralV1() { final DescribeAclsRequest original = new DescribeAclsRequest(LITERAL_FILTER, V1); diff --git a/clients/src/test/java/org/apache/kafka/common/requests/DescribeAclsResponseTest.java b/clients/src/test/java/org/apache/kafka/common/requests/DescribeAclsResponseTest.java index 81cf518536777..13a3ebb921eba 100644 --- a/clients/src/test/java/org/apache/kafka/common/requests/DescribeAclsResponseTest.java +++ b/clients/src/test/java/org/apache/kafka/common/requests/DescribeAclsResponseTest.java @@ -23,8 +23,8 @@ import org.apache.kafka.common.acl.AclPermissionType; import org.apache.kafka.common.errors.UnsupportedVersionException; import org.apache.kafka.common.protocol.types.Struct; -import org.apache.kafka.common.resource.Resource; -import org.apache.kafka.common.resource.ResourceNameType; +import org.apache.kafka.common.resource.PatternType; +import org.apache.kafka.common.resource.ResourcePattern; import org.apache.kafka.common.resource.ResourceType; import org.junit.Test; @@ -39,20 +39,28 @@ public class DescribeAclsResponseTest { private static final short V0 = 0; private static final short V1 = 1; - private static final AclBinding LITERAL_ACL1 = new AclBinding(new Resource(ResourceType.TOPIC, "foo", ResourceNameType.LITERAL), + private static final AclBinding LITERAL_ACL1 = new AclBinding(new ResourcePattern(ResourceType.TOPIC, "foo", PatternType.LITERAL), new AccessControlEntry("User:ANONYMOUS", "127.0.0.1", AclOperation.READ, AclPermissionType.DENY)); - private static final AclBinding LITERAL_ACL2 = new AclBinding(new Resource(ResourceType.GROUP, "group", ResourceNameType.LITERAL), + private static final AclBinding LITERAL_ACL2 = new AclBinding(new ResourcePattern(ResourceType.GROUP, "group", PatternType.LITERAL), new AccessControlEntry("User:*", "127.0.0.1", AclOperation.WRITE, AclPermissionType.ALLOW)); - private static final AclBinding PREFIXED_ACL1 = new AclBinding(new Resource(ResourceType.GROUP, "prefix", ResourceNameType.PREFIXED), + private static final AclBinding PREFIXED_ACL1 = new AclBinding(new ResourcePattern(ResourceType.GROUP, "prefix", PatternType.PREFIXED), new AccessControlEntry("User:*", "127.0.0.1", AclOperation.CREATE, AclPermissionType.ALLOW)); + private static final AclBinding UNKNOWN_ACL = new AclBinding(new ResourcePattern(ResourceType.UNKNOWN, "foo", PatternType.LITERAL), + new AccessControlEntry("User:ANONYMOUS", "127.0.0.1", AclOperation.READ, AclPermissionType.DENY)); + @Test(expected = UnsupportedVersionException.class) public void shouldThrowOnV0IfNotLiteral() { new DescribeAclsResponse(10, ApiError.NONE, aclBindings(PREFIXED_ACL1)).toStruct(V0); } + @Test(expected = IllegalArgumentException.class) + public void shouldThrowIfUnknown() { + new DescribeAclsResponse(10, ApiError.NONE, aclBindings(UNKNOWN_ACL)).toStruct(V0); + } + @Test public void shouldRoundTripV0() { final DescribeAclsResponse original = new DescribeAclsResponse(10, ApiError.NONE, aclBindings(LITERAL_ACL1, LITERAL_ACL2)); diff --git a/clients/src/test/java/org/apache/kafka/common/requests/ProduceRequestTest.java b/clients/src/test/java/org/apache/kafka/common/requests/ProduceRequestTest.java index ef17c96c5ad01..c45811cfab2c0 100644 --- a/clients/src/test/java/org/apache/kafka/common/requests/ProduceRequestTest.java +++ b/clients/src/test/java/org/apache/kafka/common/requests/ProduceRequestTest.java @@ -24,6 +24,7 @@ import org.apache.kafka.common.record.MemoryRecords; import org.apache.kafka.common.record.MemoryRecordsBuilder; import org.apache.kafka.common.record.RecordBatch; +import org.apache.kafka.common.record.RecordVersion; import org.apache.kafka.common.record.SimpleRecord; import org.apache.kafka.common.record.TimestampType; import org.junit.Test; @@ -31,6 +32,7 @@ import java.nio.ByteBuffer; import java.util.Collections; import java.util.HashMap; +import java.util.LinkedHashMap; import java.util.Map; import static org.junit.Assert.assertEquals; @@ -50,19 +52,19 @@ public void shouldBeFlaggedAsTransactionalWhenTransactionalRecords() throws Exce (short) 1, 1, 1, simpleRecord); final ProduceRequest request = ProduceRequest.Builder.forCurrentMagic((short) -1, 10, Collections.singletonMap(new TopicPartition("topic", 1), memoryRecords)).build(); - assertTrue(request.isTransactional()); + assertTrue(request.hasTransactionalRecords()); } @Test public void shouldNotBeFlaggedAsTransactionalWhenNoRecords() throws Exception { final ProduceRequest request = createNonIdempotentNonTransactionalRecords(); - assertFalse(request.isTransactional()); + assertFalse(request.hasTransactionalRecords()); } @Test public void shouldNotBeFlaggedAsIdempotentWhenRecordsNotIdempotent() throws Exception { final ProduceRequest request = createNonIdempotentNonTransactionalRecords(); - assertFalse(request.isTransactional()); + assertFalse(request.hasTransactionalRecords()); } @Test @@ -71,7 +73,7 @@ public void shouldBeFlaggedAsIdempotentWhenIdempotentRecords() throws Exception (short) 1, 1, 1, simpleRecord); final ProduceRequest request = ProduceRequest.Builder.forCurrentMagic((short) -1, 10, Collections.singletonMap(new TopicPartition("topic", 1), memoryRecords)).build(); - assertTrue(request.isIdempotent()); + assertTrue(request.hasIdempotentRecords()); } @Test @@ -158,6 +160,53 @@ public void testV3AndAboveCannotUseMagicV1() { assertThrowsInvalidRecordExceptionForAllVersions(requestBuilder); } + @Test + public void testMixedTransactionalData() { + final long producerId = 15L; + final short producerEpoch = 5; + final int sequence = 10; + final String transactionalId = "txnlId"; + + final MemoryRecords nonTxnRecords = MemoryRecords.withRecords(CompressionType.NONE, + new SimpleRecord("foo".getBytes())); + final MemoryRecords txnRecords = MemoryRecords.withTransactionalRecords(CompressionType.NONE, producerId, + producerEpoch, sequence, new SimpleRecord("bar".getBytes())); + + final Map recordsByPartition = new LinkedHashMap<>(); + recordsByPartition.put(new TopicPartition("foo", 0), txnRecords); + recordsByPartition.put(new TopicPartition("foo", 1), nonTxnRecords); + + final ProduceRequest.Builder builder = ProduceRequest.Builder.forMagic(RecordVersion.current().value, (short) -1, 5000, + recordsByPartition, transactionalId); + + final ProduceRequest request = builder.build(); + assertTrue(request.hasTransactionalRecords()); + assertTrue(request.hasIdempotentRecords()); + } + + @Test + public void testMixedIdempotentData() { + final long producerId = 15L; + final short producerEpoch = 5; + final int sequence = 10; + + final MemoryRecords nonTxnRecords = MemoryRecords.withRecords(CompressionType.NONE, + new SimpleRecord("foo".getBytes())); + final MemoryRecords txnRecords = MemoryRecords.withIdempotentRecords(CompressionType.NONE, producerId, + producerEpoch, sequence, new SimpleRecord("bar".getBytes())); + + final Map recordsByPartition = new LinkedHashMap<>(); + recordsByPartition.put(new TopicPartition("foo", 0), txnRecords); + recordsByPartition.put(new TopicPartition("foo", 1), nonTxnRecords); + + final ProduceRequest.Builder builder = ProduceRequest.Builder.forMagic(RecordVersion.current().value, (short) -1, 5000, + recordsByPartition, null); + + final ProduceRequest request = builder.build(); + assertFalse(request.hasTransactionalRecords()); + assertTrue(request.hasIdempotentRecords()); + } + private void assertThrowsInvalidRecordExceptionForAllVersions(ProduceRequest.Builder builder) { for (short version = builder.oldestAllowedVersion(); version < builder.latestAllowedVersion(); version++) { assertThrowsInvalidRecordException(builder, version); 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 e0c72a26f547b..6e705d2221093 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 @@ -25,6 +25,7 @@ import org.apache.kafka.common.acl.AclBindingFilter; import org.apache.kafka.common.acl.AclOperation; import org.apache.kafka.common.acl.AclPermissionType; +import org.apache.kafka.common.config.ConfigResource; import org.apache.kafka.common.errors.InvalidReplicaAssignmentException; import org.apache.kafka.common.errors.InvalidTopicException; import org.apache.kafka.common.errors.NotCoordinatorException; @@ -45,9 +46,9 @@ import org.apache.kafka.common.requests.CreateAclsResponse.AclCreationResponse; import org.apache.kafka.common.requests.DeleteAclsResponse.AclDeletionResult; import org.apache.kafka.common.requests.DeleteAclsResponse.AclFilterResponse; -import org.apache.kafka.common.resource.Resource; -import org.apache.kafka.common.resource.ResourceFilter; -import org.apache.kafka.common.resource.ResourceNameType; +import org.apache.kafka.common.resource.PatternType; +import org.apache.kafka.common.resource.ResourcePattern; +import org.apache.kafka.common.resource.ResourcePatternFilter; import org.apache.kafka.common.resource.ResourceType; import org.apache.kafka.common.security.auth.KafkaPrincipal; import org.apache.kafka.common.security.auth.SecurityProtocol; @@ -300,7 +301,7 @@ private void checkOlderFetchVersions() throws Exception { } private void verifyDescribeConfigsResponse(DescribeConfigsResponse expected, DescribeConfigsResponse actual, int version) throws Exception { - for (org.apache.kafka.common.requests.Resource resource : expected.configs().keySet()) { + for (ConfigResource resource : expected.configs().keySet()) { Collection deserializedEntries1 = actual.config(resource).entries(); Iterator expectedEntries = expected.config(resource).entries().iterator(); for (DescribeConfigsResponse.ConfigEntry entry : deserializedEntries1) { @@ -1087,23 +1088,23 @@ private TxnOffsetCommitResponse createTxnOffsetCommitResponse() { private DescribeAclsRequest createListAclsRequest() { return new DescribeAclsRequest.Builder(new AclBindingFilter( - new ResourceFilter(ResourceType.TOPIC, "mytopic", ResourceNameType.LITERAL), + new ResourcePatternFilter(ResourceType.TOPIC, "mytopic", PatternType.LITERAL), new AccessControlEntryFilter(null, null, AclOperation.ANY, AclPermissionType.ANY))).build(); } private DescribeAclsResponse createDescribeAclsResponse() { return new DescribeAclsResponse(0, ApiError.NONE, Collections.singleton(new AclBinding( - new Resource(ResourceType.TOPIC, "mytopic", ResourceNameType.LITERAL), + new ResourcePattern(ResourceType.TOPIC, "mytopic", PatternType.LITERAL), new AccessControlEntry("User:ANONYMOUS", "*", AclOperation.WRITE, AclPermissionType.ALLOW)))); } private CreateAclsRequest createCreateAclsRequest() { List creations = new ArrayList<>(); creations.add(new AclCreation(new AclBinding( - new Resource(ResourceType.TOPIC, "mytopic", ResourceNameType.LITERAL), + new ResourcePattern(ResourceType.TOPIC, "mytopic", PatternType.LITERAL), new AccessControlEntry("User:ANONYMOUS", "127.0.0.1", AclOperation.READ, AclPermissionType.ALLOW)))); creations.add(new AclCreation(new AclBinding( - new Resource(ResourceType.GROUP, "mygroup", ResourceNameType.LITERAL), + new ResourcePattern(ResourceType.GROUP, "mygroup", PatternType.LITERAL), new AccessControlEntry("User:ANONYMOUS", "*", AclOperation.WRITE, AclPermissionType.DENY)))); return new CreateAclsRequest.Builder(creations).build(); } @@ -1116,10 +1117,10 @@ private CreateAclsResponse createCreateAclsResponse() { private DeleteAclsRequest createDeleteAclsRequest() { List filters = new ArrayList<>(); filters.add(new AclBindingFilter( - new ResourceFilter(ResourceType.ANY, null, ResourceNameType.LITERAL), + new ResourcePatternFilter(ResourceType.ANY, null, PatternType.LITERAL), new AccessControlEntryFilter("User:ANONYMOUS", null, AclOperation.ANY, AclPermissionType.ANY))); filters.add(new AclBindingFilter( - new ResourceFilter(ResourceType.ANY, null, ResourceNameType.LITERAL), + new ResourcePatternFilter(ResourceType.ANY, null, PatternType.LITERAL), new AccessControlEntryFilter("User:bob", null, AclOperation.ANY, AclPermissionType.ANY))); return new DeleteAclsRequest.Builder(filters).build(); } @@ -1128,10 +1129,10 @@ private DeleteAclsResponse createDeleteAclsResponse() { List responses = new ArrayList<>(); responses.add(new AclFilterResponse(Utils.mkSet( new AclDeletionResult(new AclBinding( - new Resource(ResourceType.TOPIC, "mytopic3", ResourceNameType.LITERAL), + new ResourcePattern(ResourceType.TOPIC, "mytopic3", PatternType.LITERAL), new AccessControlEntry("User:ANONYMOUS", "*", AclOperation.DESCRIBE, AclPermissionType.ALLOW))), new AclDeletionResult(new AclBinding( - new Resource(ResourceType.TOPIC, "mytopic4", ResourceNameType.LITERAL), + new ResourcePattern(ResourceType.TOPIC, "mytopic4", PatternType.LITERAL), new AccessControlEntry("User:ANONYMOUS", "*", AclOperation.DESCRIBE, AclPermissionType.DENY)))))); responses.add(new AclFilterResponse(new ApiError(Errors.SECURITY_DISABLED, "No security"), Collections.emptySet())); @@ -1140,21 +1141,21 @@ private DeleteAclsResponse createDeleteAclsResponse() { private DescribeConfigsRequest createDescribeConfigsRequest(int version) { return new DescribeConfigsRequest.Builder(asList( - new org.apache.kafka.common.requests.Resource(org.apache.kafka.common.requests.ResourceType.BROKER, "0"), - new org.apache.kafka.common.requests.Resource(org.apache.kafka.common.requests.ResourceType.TOPIC, "topic"))) + new ConfigResource(ConfigResource.Type.BROKER, "0"), + new ConfigResource(ConfigResource.Type.TOPIC, "topic"))) .build((short) version); } private DescribeConfigsRequest createDescribeConfigsRequestWithConfigEntries(int version) { - Map> resources = new HashMap<>(); - resources.put(new org.apache.kafka.common.requests.Resource(org.apache.kafka.common.requests.ResourceType.BROKER, "0"), asList("foo", "bar")); - resources.put(new org.apache.kafka.common.requests.Resource(org.apache.kafka.common.requests.ResourceType.TOPIC, "topic"), null); - resources.put(new org.apache.kafka.common.requests.Resource(org.apache.kafka.common.requests.ResourceType.TOPIC, "topic a"), Collections.emptyList()); + Map> resources = new HashMap<>(); + resources.put(new ConfigResource(ConfigResource.Type.BROKER, "0"), asList("foo", "bar")); + resources.put(new ConfigResource(ConfigResource.Type.TOPIC, "topic"), null); + resources.put(new ConfigResource(ConfigResource.Type.TOPIC, "topic a"), Collections.emptyList()); return new DescribeConfigsRequest.Builder(resources).build((short) version); } private DescribeConfigsResponse createDescribeConfigsResponse() { - Map configs = new HashMap<>(); + Map configs = new HashMap<>(); List synonyms = Collections.emptyList(); List configEntries = asList( new DescribeConfigsResponse.ConfigEntry("config_name", "config_value", @@ -1162,29 +1163,29 @@ private DescribeConfigsResponse createDescribeConfigsResponse() { new DescribeConfigsResponse.ConfigEntry("another_name", "another value", DescribeConfigsResponse.ConfigSource.DEFAULT_CONFIG, false, true, synonyms) ); - configs.put(new org.apache.kafka.common.requests.Resource(org.apache.kafka.common.requests.ResourceType.BROKER, "0"), new DescribeConfigsResponse.Config( + configs.put(new ConfigResource(ConfigResource.Type.BROKER, "0"), new DescribeConfigsResponse.Config( ApiError.NONE, configEntries)); - configs.put(new org.apache.kafka.common.requests.Resource(org.apache.kafka.common.requests.ResourceType.TOPIC, "topic"), new DescribeConfigsResponse.Config( + configs.put(new ConfigResource(ConfigResource.Type.TOPIC, "topic"), new DescribeConfigsResponse.Config( ApiError.NONE, Collections.emptyList())); return new DescribeConfigsResponse(200, configs); } private AlterConfigsRequest createAlterConfigsRequest() { - Map configs = new HashMap<>(); + Map configs = new HashMap<>(); List configEntries = asList( new AlterConfigsRequest.ConfigEntry("config_name", "config_value"), new AlterConfigsRequest.ConfigEntry("another_name", "another value") ); - configs.put(new org.apache.kafka.common.requests.Resource(org.apache.kafka.common.requests.ResourceType.BROKER, "0"), new AlterConfigsRequest.Config(configEntries)); - configs.put(new org.apache.kafka.common.requests.Resource(org.apache.kafka.common.requests.ResourceType.TOPIC, "topic"), + configs.put(new ConfigResource(ConfigResource.Type.BROKER, "0"), new AlterConfigsRequest.Config(configEntries)); + configs.put(new ConfigResource(ConfigResource.Type.TOPIC, "topic"), new AlterConfigsRequest.Config(Collections.emptyList())); return new AlterConfigsRequest((short) 0, configs, false); } private AlterConfigsResponse createAlterConfigsResponse() { - Map errors = new HashMap<>(); - errors.put(new org.apache.kafka.common.requests.Resource(org.apache.kafka.common.requests.ResourceType.BROKER, "0"), ApiError.NONE); - errors.put(new org.apache.kafka.common.requests.Resource(org.apache.kafka.common.requests.ResourceType.TOPIC, "topic"), new ApiError(Errors.INVALID_REQUEST, "This request is invalid")); + Map errors = new HashMap<>(); + errors.put(new ConfigResource(ConfigResource.Type.BROKER, "0"), ApiError.NONE); + errors.put(new ConfigResource(ConfigResource.Type.TOPIC, "topic"), new ApiError(Errors.INVALID_REQUEST, "This request is invalid")); return new AlterConfigsResponse(20, errors); } diff --git a/clients/src/test/java/org/apache/kafka/common/resource/ResourceFilterTest.java b/clients/src/test/java/org/apache/kafka/common/resource/ResourceFilterTest.java index 9b2d6d4e696ff..4399744d91d64 100644 --- a/clients/src/test/java/org/apache/kafka/common/resource/ResourceFilterTest.java +++ b/clients/src/test/java/org/apache/kafka/common/resource/ResourceFilterTest.java @@ -19,145 +19,64 @@ import org.junit.Test; -import static org.apache.kafka.common.resource.ResourceNameType.LITERAL; -import static org.apache.kafka.common.resource.ResourceNameType.PREFIXED; import static org.apache.kafka.common.resource.ResourceType.ANY; import static org.apache.kafka.common.resource.ResourceType.GROUP; import static org.apache.kafka.common.resource.ResourceType.TOPIC; -import static org.apache.kafka.common.resource.ResourceType.UNKNOWN; import static org.junit.Assert.assertFalse; import static org.junit.Assert.assertTrue; public class ResourceFilterTest { - @Test(expected = IllegalArgumentException.class) - public void shouldThrowIfResourceTypeIsAny() { - new ResourceFilter(ANY, null, ResourceNameType.ANY) - .matches(new Resource(ANY, "Name", PREFIXED)); - } - - @Test(expected = IllegalArgumentException.class) - public void shouldThrowIfResourceTypeIsUnknown() { - new ResourceFilter(ANY, null, ResourceNameType.ANY) - .matches(new Resource(UNKNOWN, "Name", LITERAL)); - } - - @Test(expected = IllegalArgumentException.class) - public void shouldThrowIfResourceNameTypeIsAny() { - new ResourceFilter(ANY, null, ResourceNameType.ANY) - .matches(new Resource(GROUP, "Name", ResourceNameType.ANY)); - } - - @Test(expected = IllegalArgumentException.class) - public void shouldThrowIfAclResourceNameTypeIsUnknown() { - new ResourceFilter(ANY, null, ResourceNameType.ANY) - .matches(new Resource(GROUP, "Name", ResourceNameType.UNKNOWN)); - } - @Test public void shouldNotMatchIfDifferentResourceType() { - assertFalse(new ResourceFilter(TOPIC, "Name", LITERAL) - .matches(new Resource(GROUP, "Name", LITERAL))); + assertFalse(new ResourceFilter(TOPIC, "Name") + .matches(new Resource(GROUP, "Name"))); } @Test public void shouldNotMatchIfDifferentName() { - assertFalse(new ResourceFilter(TOPIC, "Different", PREFIXED) - .matches(new Resource(TOPIC, "Name", PREFIXED))); + assertFalse(new ResourceFilter(TOPIC, "Different") + .matches(new Resource(TOPIC, "Name"))); } @Test public void shouldNotMatchIfDifferentNameCase() { - assertFalse(new ResourceFilter(TOPIC, "NAME", LITERAL) - .matches(new Resource(TOPIC, "Name", LITERAL))); - } - - @Test - public void shouldNotMatchIfDifferentNameType() { - assertFalse(new ResourceFilter(TOPIC, "Name", LITERAL) - .matches(new Resource(TOPIC, "Name", PREFIXED))); + assertFalse(new ResourceFilter(TOPIC, "NAME") + .matches(new Resource(TOPIC, "Name"))); } @Test public void shouldMatchWhereResourceTypeIsAny() { - assertTrue(new ResourceFilter(ANY, "Name", PREFIXED) - .matches(new Resource(TOPIC, "Name", PREFIXED))); + assertTrue(new ResourceFilter(ANY, "Name") + .matches(new Resource(TOPIC, "Name"))); } @Test public void shouldMatchWhereResourceNameIsAny() { - assertTrue(new ResourceFilter(TOPIC, null, PREFIXED) - .matches(new Resource(TOPIC, "Name", PREFIXED))); - } - - @Test - public void shouldMatchWhereResourceNameTypeIsAny() { - assertTrue(new ResourceFilter(TOPIC, null, ResourceNameType.ANY) - .matches(new Resource(TOPIC, "Name", PREFIXED))); + assertTrue(new ResourceFilter(TOPIC, null) + .matches(new Resource(TOPIC, "Name"))); } @Test - public void shouldMatchLiteralIfExactMatch() { - assertTrue(new ResourceFilter(TOPIC, "Name", LITERAL) - .matches(new Resource(TOPIC, "Name", LITERAL))); + public void shouldMatchIfExactMatch() { + assertTrue(new ResourceFilter(TOPIC, "Name") + .matches(new Resource(TOPIC, "Name"))); } @Test - public void shouldMatchLiteralIfNameMatchesAndFilterIsOnAnyNameType() { - assertTrue(new ResourceFilter(TOPIC, "Name", ResourceNameType.ANY) - .matches(new Resource(TOPIC, "Name", LITERAL))); + public void shouldMatchWildcardIfExactMatch() { + assertTrue(new ResourceFilter(TOPIC, "*") + .matches(new Resource(TOPIC, "*"))); } @Test - public void shouldNotMatchLiteralIfNamePrefixed() { - assertFalse(new ResourceFilter(TOPIC, "Name-something", ResourceNameType.ANY) - .matches(new Resource(TOPIC, "Name", LITERAL))); - } - - @Test - public void shouldMatchLiteralWildcardIfExactMatch() { - assertTrue(new ResourceFilter(TOPIC, "*", LITERAL) - .matches(new Resource(TOPIC, "*", LITERAL))); - } - - @Test - public void shouldNotMatchLiteralWildcardAgainstOtherName() { - assertFalse(new ResourceFilter(TOPIC, "Name", LITERAL) - .matches(new Resource(TOPIC, "*", LITERAL))); + public void shouldNotMatchWildcardAgainstOtherName() { + assertFalse(new ResourceFilter(TOPIC, "Name") + .matches(new Resource(TOPIC, "*"))); } @Test public void shouldNotMatchLiteralWildcardTheWayAround() { - assertFalse(new ResourceFilter(TOPIC, "*", LITERAL) - .matches(new Resource(TOPIC, "Name", LITERAL))); - } - - @Test - public void shouldMatchLiteralWildcardIfFilterHasNameTypeOfAny() { - assertTrue(new ResourceFilter(TOPIC, "Name", ResourceNameType.ANY) - .matches(new Resource(TOPIC, "*", LITERAL))); - } - - @Test - public void shouldMatchPrefixedIfExactMatch() { - assertTrue(new ResourceFilter(TOPIC, "Name", PREFIXED) - .matches(new Resource(TOPIC, "Name", PREFIXED))); - } - - @Test - public void shouldNotMatchIfBothPrefixedAndFilterIsPrefixOfResource() { - assertFalse(new ResourceFilter(TOPIC, "Name", PREFIXED) - .matches(new Resource(TOPIC, "Name-something", PREFIXED))); - } - - @Test - public void shouldNotMatchIfBothPrefixedAndResourceIsPrefixOfFilter() { - assertFalse(new ResourceFilter(TOPIC, "Name-something", PREFIXED) - .matches(new Resource(TOPIC, "Name", PREFIXED))); - } - - @Test - public void shouldMatchPrefixedIfNamePrefixedAnyFilterTypeIsAny() { - assertTrue(new ResourceFilter(TOPIC, "Name-something", ResourceNameType.ANY) - .matches(new Resource(TOPIC, "Name", PREFIXED))); + assertFalse(new ResourceFilter(TOPIC, "*") + .matches(new Resource(TOPIC, "Name"))); } } \ No newline at end of file diff --git a/clients/src/test/java/org/apache/kafka/common/security/oauthbearer/internals/OAuthBearerClientInitialResponseTest.java b/clients/src/test/java/org/apache/kafka/common/security/oauthbearer/internals/OAuthBearerClientInitialResponseTest.java new file mode 100644 index 0000000000000..eccf2dd2ed426 --- /dev/null +++ b/clients/src/test/java/org/apache/kafka/common/security/oauthbearer/internals/OAuthBearerClientInitialResponseTest.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.security.oauthbearer.internals; + +import static org.junit.Assert.assertEquals; + +import org.junit.Test; + +import java.nio.charset.StandardCharsets; + +public class OAuthBearerClientInitialResponseTest { + + @Test + public void testToken() throws Exception { + String message = "n,,\u0001auth=Bearer 123.345.567\u0001\u0001"; + OAuthBearerClientInitialResponse response = new OAuthBearerClientInitialResponse(message.getBytes(StandardCharsets.UTF_8)); + assertEquals("123.345.567", response.tokenValue()); + assertEquals("", response.authorizationId()); + } + + @Test + public void testAuthorizationId() throws Exception { + String message = "n,a=myuser,\u0001auth=Bearer 345\u0001\u0001"; + OAuthBearerClientInitialResponse response = new OAuthBearerClientInitialResponse(message.getBytes(StandardCharsets.UTF_8)); + assertEquals("345", response.tokenValue()); + assertEquals("myuser", response.authorizationId()); + } + + @Test + public void testProperties() throws Exception { + String message = "n,,\u0001propA=valueA1, valueA2\u0001auth=Bearer 567\u0001propB=valueB\u0001\u0001"; + OAuthBearerClientInitialResponse response = new OAuthBearerClientInitialResponse(message.getBytes(StandardCharsets.UTF_8)); + assertEquals("567", response.tokenValue()); + assertEquals("", response.authorizationId()); + assertEquals("valueA1, valueA2", response.propertyValue("propA")); + assertEquals("valueB", response.propertyValue("propB")); + } + + // The example in the RFC uses `vF9dft4qmTc2Nvb3RlckBhbHRhdmlzdGEuY29tCg==` as the token + // But since we use Base64Url encoding, padding is omitted. Hence this test verifies without '='. + @Test + public void testRfc7688Example() throws Exception { + String message = "n,a=user@example.com,\u0001host=server.example.com\u0001port=143\u0001" + + "auth=Bearer vF9dft4qmTc2Nvb3RlckBhbHRhdmlzdGEuY29tCg\u0001\u0001"; + OAuthBearerClientInitialResponse response = new OAuthBearerClientInitialResponse(message.getBytes(StandardCharsets.UTF_8)); + assertEquals("vF9dft4qmTc2Nvb3RlckBhbHRhdmlzdGEuY29tCg", response.tokenValue()); + assertEquals("user@example.com", response.authorizationId()); + assertEquals("server.example.com", response.propertyValue("host")); + assertEquals("143", response.propertyValue("port")); + } +} diff --git a/clients/src/test/java/org/apache/kafka/common/security/oauthbearer/internals/OAuthBearerSaslServerTest.java b/clients/src/test/java/org/apache/kafka/common/security/oauthbearer/internals/OAuthBearerSaslServerTest.java index bf21f2b343099..6b53e963af74f 100644 --- a/clients/src/test/java/org/apache/kafka/common/security/oauthbearer/internals/OAuthBearerSaslServerTest.java +++ b/clients/src/test/java/org/apache/kafka/common/security/oauthbearer/internals/OAuthBearerSaslServerTest.java @@ -75,39 +75,41 @@ public void setUp() throws Exception { @Test public void noAuthorizationIdSpecified() throws Exception { byte[] nextChallenge = saslServer - .evaluateResponse(clientInitialResponseText(null).getBytes(StandardCharsets.UTF_8)); + .evaluateResponse(clientInitialResponse(null)); assertTrue("Next challenge is not empty", nextChallenge.length == 0); } @Test public void authorizatonIdEqualsAuthenticationId() throws Exception { byte[] nextChallenge = saslServer - .evaluateResponse(clientInitialResponseText(USER).getBytes(StandardCharsets.UTF_8)); + .evaluateResponse(clientInitialResponse(USER)); assertTrue("Next challenge is not empty", nextChallenge.length == 0); } @Test(expected = SaslAuthenticationException.class) public void authorizatonIdNotEqualsAuthenticationId() throws Exception { - saslServer.evaluateResponse(clientInitialResponseText(USER + "x").getBytes(StandardCharsets.UTF_8)); + saslServer.evaluateResponse(clientInitialResponse(USER + "x")); } @Test public void illegalToken() throws Exception { - byte[] bytes = saslServer - .evaluateResponse((clientInitialResponseText(null) + "AB").getBytes(StandardCharsets.UTF_8)); + byte[] bytes = saslServer.evaluateResponse(clientInitialResponse(null, true)); String challenge = new String(bytes, StandardCharsets.UTF_8); assertEquals("{\"status\":\"invalid_token\"}", challenge); } - private String clientInitialResponseText(String authorizationId) + private byte[] clientInitialResponse(String authorizationId) + throws OAuthBearerConfigException, IOException, UnsupportedCallbackException, LoginException { + return clientInitialResponse(authorizationId, false); + } + + private byte[] clientInitialResponse(String authorizationId, boolean illegalToken) throws OAuthBearerConfigException, IOException, UnsupportedCallbackException, LoginException { OAuthBearerTokenCallback callback = new OAuthBearerTokenCallback(); LOGIN_CALLBACK_HANDLER.handle(new Callback[] {callback}); OAuthBearerToken token = callback.token(); String compactSerialization = token.value(); - String clientInitialResponseText = "n," - + (authorizationId == null || authorizationId.isEmpty() ? "" : "a=" + authorizationId) + ",auth=Bearer " - + compactSerialization; - return clientInitialResponseText; + String tokenValue = compactSerialization + (illegalToken ? "AB" : ""); + return new OAuthBearerClientInitialResponse(tokenValue, authorizationId, Collections.emptyMap()).toBytes(); } } diff --git a/connect/api/src/main/java/org/apache/kafka/connect/connector/ConnectRecord.java b/connect/api/src/main/java/org/apache/kafka/connect/connector/ConnectRecord.java index 2ad8a046d768a..4aa7d5a742fa2 100644 --- a/connect/api/src/main/java/org/apache/kafka/connect/connector/ConnectRecord.java +++ b/connect/api/src/main/java/org/apache/kafka/connect/connector/ConnectRecord.java @@ -140,7 +140,9 @@ public String toString() { "topic='" + topic + '\'' + ", kafkaPartition=" + kafkaPartition + ", key=" + key + + ", keySchema=" + keySchema + ", value=" + value + + ", valueSchema=" + valueSchema + ", timestamp=" + timestamp + ", headers=" + headers + '}'; diff --git a/connect/api/src/main/java/org/apache/kafka/connect/data/ConnectSchema.java b/connect/api/src/main/java/org/apache/kafka/connect/data/ConnectSchema.java index ff8271635f304..f1a05bb19a6f0 100644 --- a/connect/api/src/main/java/org/apache/kafka/connect/data/ConnectSchema.java +++ b/connect/api/src/main/java/org/apache/kafka/connect/data/ConnectSchema.java @@ -291,7 +291,7 @@ public boolean equals(Object o) { Objects.equals(name, schema.name) && Objects.equals(doc, schema.doc) && Objects.equals(type, schema.type) && - Objects.equals(defaultValue, schema.defaultValue) && + Objects.deepEquals(defaultValue, schema.defaultValue) && Objects.equals(fields, schema.fields) && Objects.equals(keySchema, schema.keySchema) && Objects.equals(valueSchema, schema.valueSchema) && diff --git a/connect/api/src/main/java/org/apache/kafka/connect/data/SchemaProjector.java b/connect/api/src/main/java/org/apache/kafka/connect/data/SchemaProjector.java index ea31752f60e58..5400705cd105c 100644 --- a/connect/api/src/main/java/org/apache/kafka/connect/data/SchemaProjector.java +++ b/connect/api/src/main/java/org/apache/kafka/connect/data/SchemaProjector.java @@ -160,7 +160,7 @@ private static Object projectPrimitive(Schema source, Object record, Schema targ assert source.type().isPrimitive(); assert target.type().isPrimitive(); Object result; - if (isPromotable(source.type(), target.type())) { + if (isPromotable(source.type(), target.type()) && record instanceof Number) { Number numberRecord = (Number) record; switch (target.type()) { case INT8: diff --git a/connect/api/src/main/java/org/apache/kafka/connect/data/Values.java b/connect/api/src/main/java/org/apache/kafka/connect/data/Values.java index d643aa22ddd57..bf0c528c12178 100644 --- a/connect/api/src/main/java/org/apache/kafka/connect/data/Values.java +++ b/connect/api/src/main/java/org/apache/kafka/connect/data/Values.java @@ -69,7 +69,7 @@ public class Values { private static final String FALSE_LITERAL = Boolean.TRUE.toString(); private static final long MILLIS_PER_DAY = 24 * 60 * 60 * 1000; private static final String NULL_VALUE = "null"; - private static final String ISO_8601_DATE_FORMAT_PATTERN = "YYYY-MM-DD"; + private static final String ISO_8601_DATE_FORMAT_PATTERN = "yyyy-MM-dd"; private static final String ISO_8601_TIME_FORMAT_PATTERN = "HH:mm:ss.SSS'Z'"; private static final String ISO_8601_TIMESTAMP_FORMAT_PATTERN = ISO_8601_DATE_FORMAT_PATTERN + "'T'" + ISO_8601_TIME_FORMAT_PATTERN; @@ -707,7 +707,7 @@ protected static String escape(String value) { return value.replaceAll("\\\\", "\\\\\\\\").replaceAll("\"", "\\\\\""); } - protected static DateFormat dateFormatFor(java.util.Date value) { + public static DateFormat dateFormatFor(java.util.Date value) { if (value.getTime() < MILLIS_PER_DAY) { return new SimpleDateFormat(ISO_8601_TIME_FORMAT_PATTERN); } diff --git a/connect/api/src/main/java/org/apache/kafka/connect/header/ConnectHeaders.java b/connect/api/src/main/java/org/apache/kafka/connect/header/ConnectHeaders.java index 185ba65bf0fb6..0b5c484b3533e 100644 --- a/connect/api/src/main/java/org/apache/kafka/connect/header/ConnectHeaders.java +++ b/connect/api/src/main/java/org/apache/kafka/connect/header/ConnectHeaders.java @@ -274,7 +274,7 @@ public Iterator

iterator() { @Override public Headers remove(String key) { checkKey(key); - if (!headers.isEmpty()) { + if (!isEmpty()) { Iterator
iterator = iterator(); while (iterator.hasNext()) { if (iterator.next().key().equals(key)) { @@ -287,7 +287,7 @@ public Headers remove(String key) { @Override public Headers retainLatest() { - if (!headers.isEmpty()) { + if (!isEmpty()) { Set keys = new HashSet<>(); ListIterator
iter = headers.listIterator(headers.size()); while (iter.hasPrevious()) { @@ -304,7 +304,7 @@ public Headers retainLatest() { @Override public Headers retainLatest(String key) { checkKey(key); - if (!headers.isEmpty()) { + if (!isEmpty()) { boolean found = false; ListIterator
iter = headers.listIterator(headers.size()); while (iter.hasPrevious()) { @@ -322,7 +322,7 @@ public Headers retainLatest(String key) { @Override public Headers apply(String key, HeaderTransform transform) { checkKey(key); - if (!headers.isEmpty()) { + if (!isEmpty()) { ListIterator
iter = headers.listIterator(); while (iter.hasNext()) { Header orig = iter.next(); @@ -341,7 +341,7 @@ public Headers apply(String key, HeaderTransform transform) { @Override public Headers apply(HeaderTransform transform) { - if (!headers.isEmpty()) { + if (!isEmpty()) { ListIterator
iter = headers.listIterator(); while (iter.hasNext()) { Header orig = iter.next(); diff --git a/connect/api/src/main/java/org/apache/kafka/connect/health/AbstractState.java b/connect/api/src/main/java/org/apache/kafka/connect/health/AbstractState.java index a18c46334cf29..f707b3c3026ed 100644 --- a/connect/api/src/main/java/org/apache/kafka/connect/health/AbstractState.java +++ b/connect/api/src/main/java/org/apache/kafka/connect/health/AbstractState.java @@ -17,6 +17,8 @@ package org.apache.kafka.connect.health; +import java.util.Objects; + /** * Provides the current status along with identifier for Connect worker and tasks. */ @@ -34,10 +36,10 @@ public abstract class AbstractState { * @param traceMessage any error trace message associated with the connector or the task; may be null or empty */ public AbstractState(String state, String workerId, String traceMessage) { - if (state != null && !state.trim().isEmpty()) { + if (state == null || state.trim().isEmpty()) { throw new IllegalArgumentException("State must not be null or empty"); } - if (workerId != null && !workerId.trim().isEmpty()) { + if (workerId == null || workerId.trim().isEmpty()) { throw new IllegalArgumentException("Worker ID must not be null or empty"); } this.state = state; @@ -71,4 +73,21 @@ public String workerId() { public String traceMessage() { return traceMessage; } + + @Override + public boolean equals(Object o) { + if (this == o) + return true; + if (o == null || getClass() != o.getClass()) + return false; + AbstractState that = (AbstractState) o; + return state.equals(that.state) + && Objects.equals(traceMessage, that.traceMessage) + && workerId.equals(that.workerId); + } + + @Override + public int hashCode() { + return Objects.hash(state, traceMessage, workerId); + } } diff --git a/connect/api/src/main/java/org/apache/kafka/connect/health/ConnectorHealth.java b/connect/api/src/main/java/org/apache/kafka/connect/health/ConnectorHealth.java index 3a9efd15372ed..12fa6b76aff1e 100644 --- a/connect/api/src/main/java/org/apache/kafka/connect/health/ConnectorHealth.java +++ b/connect/api/src/main/java/org/apache/kafka/connect/health/ConnectorHealth.java @@ -35,7 +35,7 @@ public ConnectorHealth(String name, ConnectorState connectorState, Map tasks, ConnectorType type) { - if (name != null && !name.trim().isEmpty()) { + if (name == null || name.trim().isEmpty()) { throw new IllegalArgumentException("Connector name is required"); } Objects.requireNonNull(connectorState, "connectorState can't be null"); @@ -83,4 +83,31 @@ public ConnectorType type() { return type; } + @Override + public boolean equals(Object o) { + if (this == o) + return true; + if (o == null || getClass() != o.getClass()) + return false; + ConnectorHealth that = (ConnectorHealth) o; + return name.equals(that.name) + && connectorState.equals(that.connectorState) + && tasks.equals(that.tasks) + && type == that.type; + } + + @Override + public int hashCode() { + return Objects.hash(name, connectorState, tasks, type); + } + + @Override + public String toString() { + return "ConnectorHealth{" + + "name='" + name + '\'' + + ", connectorState=" + connectorState + + ", tasks=" + tasks + + ", type=" + type + + '}'; + } } diff --git a/connect/api/src/main/java/org/apache/kafka/connect/health/ConnectorState.java b/connect/api/src/main/java/org/apache/kafka/connect/health/ConnectorState.java index d5571bc4ff5d0..63044265bb999 100644 --- a/connect/api/src/main/java/org/apache/kafka/connect/health/ConnectorState.java +++ b/connect/api/src/main/java/org/apache/kafka/connect/health/ConnectorState.java @@ -32,4 +32,13 @@ public class ConnectorState extends AbstractState { public ConnectorState(String state, String workerId, String traceMessage) { super(state, workerId, traceMessage); } + + @Override + public String toString() { + return "ConnectorState{" + + "state='" + state() + '\'' + + ", traceMessage='" + traceMessage() + '\'' + + ", workerId='" + workerId() + '\'' + + '}'; + } } diff --git a/connect/api/src/main/java/org/apache/kafka/connect/health/TaskState.java b/connect/api/src/main/java/org/apache/kafka/connect/health/TaskState.java index 1c1be159970d9..ae78a5f3af990 100644 --- a/connect/api/src/main/java/org/apache/kafka/connect/health/TaskState.java +++ b/connect/api/src/main/java/org/apache/kafka/connect/health/TaskState.java @@ -50,20 +50,28 @@ public int taskId() { @Override public boolean equals(Object o) { - if (this == o) { + if (this == o) return true; - } - if (o == null || getClass() != o.getClass()) { + if (o == null || getClass() != o.getClass()) + return false; + if (!super.equals(o)) return false; - } - TaskState taskState = (TaskState) o; - return taskId == taskState.taskId; } @Override public int hashCode() { - return Objects.hash(taskId); + return Objects.hash(super.hashCode(), taskId); + } + + @Override + public String toString() { + return "TaskState{" + + "taskId='" + taskId + '\'' + + "state='" + state() + '\'' + + ", traceMessage='" + traceMessage() + '\'' + + ", workerId='" + workerId() + '\'' + + '}'; } } diff --git a/connect/api/src/test/java/org/apache/kafka/connect/data/ConnectSchemaTest.java b/connect/api/src/test/java/org/apache/kafka/connect/data/ConnectSchemaTest.java index 339ef23ca54e3..048784e3335d6 100644 --- a/connect/api/src/test/java/org/apache/kafka/connect/data/ConnectSchemaTest.java +++ b/connect/api/src/test/java/org/apache/kafka/connect/data/ConnectSchemaTest.java @@ -268,6 +268,16 @@ public void testArrayEquality() { assertNotEquals(s1, differentValueSchema); } + @Test + public void testArrayDefaultValueEquality() { + ConnectSchema s1 = new ConnectSchema(Schema.Type.ARRAY, false, new String[] {"a", "b"}, null, null, null, null, null, null, SchemaBuilder.int8().build()); + ConnectSchema s2 = new ConnectSchema(Schema.Type.ARRAY, false, new String[] {"a", "b"}, null, null, null, null, null, null, SchemaBuilder.int8().build()); + ConnectSchema differentValueSchema = new ConnectSchema(Schema.Type.ARRAY, false, new String[] {"b", "c"}, null, null, null, null, null, null, SchemaBuilder.int8().build()); + + assertEquals(s1, s2); + assertNotEquals(s1, differentValueSchema); + } + @Test public void testMapEquality() { // Same as testArrayEquality, but for both key and value schemas diff --git a/connect/api/src/test/java/org/apache/kafka/connect/data/SchemaProjectorTest.java b/connect/api/src/test/java/org/apache/kafka/connect/data/SchemaProjectorTest.java index 151114e2d2937..0db4eecf25cf9 100644 --- a/connect/api/src/test/java/org/apache/kafka/connect/data/SchemaProjectorTest.java +++ b/connect/api/src/test/java/org/apache/kafka/connect/data/SchemaProjectorTest.java @@ -352,6 +352,17 @@ public void testLogicalTypeProjection() throws Exception { projected = SchemaProjector.project(Timestamp.SCHEMA, 34567L, Timestamp.SCHEMA); assertEquals(34567L, projected); + java.util.Date date = new java.util.Date(); + + projected = SchemaProjector.project(Date.SCHEMA, date, Date.SCHEMA); + assertEquals(date, projected); + + projected = SchemaProjector.project(Time.SCHEMA, date, Time.SCHEMA); + assertEquals(date, projected); + + projected = SchemaProjector.project(Timestamp.SCHEMA, date, Timestamp.SCHEMA); + assertEquals(date, projected); + Schema namedSchema = SchemaBuilder.int32().name("invalidLogicalTypeName").build(); for (Schema logicalTypeSchema: logicalTypeSchemas) { try { diff --git a/connect/api/src/test/java/org/apache/kafka/connect/header/ConnectHeadersTest.java b/connect/api/src/test/java/org/apache/kafka/connect/header/ConnectHeadersTest.java index 343bc5d43f59a..72418ba47ffa7 100644 --- a/connect/api/src/test/java/org/apache/kafka/connect/header/ConnectHeadersTest.java +++ b/connect/api/src/test/java/org/apache/kafka/connect/header/ConnectHeadersTest.java @@ -118,6 +118,14 @@ public void shouldHaveToString() { assertNotNull(headers.toString()); } + @Test + public void shouldRetainLatestWhenEmpty() { + headers.retainLatest(other); + headers.retainLatest(key); + headers.retainLatest(); + assertTrue(headers.isEmpty()); + } + @Test public void shouldAddMultipleHeadersWithSameKeyAndRetainLatest() { populate(headers); @@ -179,6 +187,12 @@ public void shouldNotAddHeadersWithObjectValuesAndMismatchedSchema() { attemptAndFailToAddHeader("k2", Schema.OPTIONAL_STRING_SCHEMA, 0L); } + @Test + public void shouldRemoveAllHeadersWithSameKeyWhenEmpty() { + headers.remove(key); + assertNoHeaderWithKey(key); + } + @Test public void shouldRemoveAllHeadersWithSameKey() { populate(headers); @@ -211,6 +225,13 @@ public void shouldRemoveAllHeaders() { assertTrue(headers.isEmpty()); } + @Test + public void shouldTransformHeadersWhenEmpty() { + headers.apply(appendToKey("-suffix")); + headers.apply(key, appendToKey("-suffix")); + assertTrue(headers.isEmpty()); + } + @Test public void shouldTransformHeaders() { populate(headers); @@ -544,4 +565,4 @@ protected void assertHeader(Header header, String key, Schema schema, Object val assertSame(schema, header.schema()); assertSame(value, header.value()); } -} \ No newline at end of file +} diff --git a/connect/basic-auth-extension/src/main/java/org/apache/kafka/connect/rest/basic/auth/extenstion/BasicAuthSecurityRestExtension.java b/connect/basic-auth-extension/src/main/java/org/apache/kafka/connect/rest/basic/auth/extension/BasicAuthSecurityRestExtension.java similarity index 94% rename from connect/basic-auth-extension/src/main/java/org/apache/kafka/connect/rest/basic/auth/extenstion/BasicAuthSecurityRestExtension.java rename to connect/basic-auth-extension/src/main/java/org/apache/kafka/connect/rest/basic/auth/extension/BasicAuthSecurityRestExtension.java index 91d5d9ca00cc4..4169e9eaea652 100644 --- a/connect/basic-auth-extension/src/main/java/org/apache/kafka/connect/rest/basic/auth/extenstion/BasicAuthSecurityRestExtension.java +++ b/connect/basic-auth-extension/src/main/java/org/apache/kafka/connect/rest/basic/auth/extension/BasicAuthSecurityRestExtension.java @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.kafka.connect.rest.basic.auth.extenstion; +package org.apache.kafka.connect.rest.basic.auth.extension; import org.apache.kafka.common.utils.AppInfoParser; import org.apache.kafka.connect.rest.ConnectRestExtension; @@ -33,13 +33,13 @@ * *

To use this extension, one needs to add the following config in the {@code worker.properties} *

- *     rest.extension.classes = org.apache.kafka.connect.rest.basic.auth.extenstion.BasicAuthSecurityRestExtension
+ *     rest.extension.classes = org.apache.kafka.connect.rest.basic.auth.extension.BasicAuthSecurityRestExtension
  * 
* *

An example JAAS config would look as below *

  *         KafkaConnect {
- *              org.apache.kafka.connect.rest.basic.auth.extenstion.PropertyFileLoginModule required
+ *              org.apache.kafka.connect.rest.basic.auth.extension.PropertyFileLoginModule required
  *              file="/mnt/secret/credentials.properties";
  *         };
  *
diff --git a/connect/basic-auth-extension/src/main/java/org/apache/kafka/connect/rest/basic/auth/extenstion/JaasBasicAuthFilter.java b/connect/basic-auth-extension/src/main/java/org/apache/kafka/connect/rest/basic/auth/extension/JaasBasicAuthFilter.java similarity index 85% rename from connect/basic-auth-extension/src/main/java/org/apache/kafka/connect/rest/basic/auth/extenstion/JaasBasicAuthFilter.java rename to connect/basic-auth-extension/src/main/java/org/apache/kafka/connect/rest/basic/auth/extension/JaasBasicAuthFilter.java index 7231700af7c5a..d5b15c6c65a5f 100644 --- a/connect/basic-auth-extension/src/main/java/org/apache/kafka/connect/rest/basic/auth/extenstion/JaasBasicAuthFilter.java +++ b/connect/basic-auth-extension/src/main/java/org/apache/kafka/connect/rest/basic/auth/extension/JaasBasicAuthFilter.java @@ -15,8 +15,10 @@ * limitations under the License. */ -package org.apache.kafka.connect.rest.basic.auth.extenstion; +package org.apache.kafka.connect.rest.basic.auth.extension; +import java.util.regex.Pattern; +import javax.ws.rs.HttpMethod; import org.apache.kafka.common.config.ConfigException; import java.io.IOException; @@ -35,18 +37,18 @@ import javax.ws.rs.core.Response; public class JaasBasicAuthFilter implements ContainerRequestFilter { - private static final String CONNECT_LOGIN_MODULE = "KafkaConnect"; static final String AUTHORIZATION = "Authorization"; - + private static final Pattern TASK_REQUEST_PATTERN = Pattern.compile("/?connectors/([^/]+)/tasks/?"); @Override public void filter(ContainerRequestContext requestContext) throws IOException { - try { - LoginContext loginContext = - new LoginContext(CONNECT_LOGIN_MODULE, new BasicAuthCallBackHandler( - requestContext.getHeaderString(AUTHORIZATION))); - loginContext.login(); + if (!(requestContext.getMethod().equals(HttpMethod.POST) && TASK_REQUEST_PATTERN.matcher(requestContext.getUriInfo().getPath()).matches())) { + LoginContext loginContext = + new LoginContext(CONNECT_LOGIN_MODULE, new BasicAuthCallBackHandler( + requestContext.getHeaderString(AUTHORIZATION))); + loginContext.login(); + } } catch (LoginException | ConfigException e) { requestContext.abortWith( Response.status(Response.Status.UNAUTHORIZED) diff --git a/connect/basic-auth-extension/src/main/java/org/apache/kafka/connect/rest/basic/auth/extenstion/PropertyFileLoginModule.java b/connect/basic-auth-extension/src/main/java/org/apache/kafka/connect/rest/basic/auth/extension/PropertyFileLoginModule.java similarity index 98% rename from connect/basic-auth-extension/src/main/java/org/apache/kafka/connect/rest/basic/auth/extenstion/PropertyFileLoginModule.java rename to connect/basic-auth-extension/src/main/java/org/apache/kafka/connect/rest/basic/auth/extension/PropertyFileLoginModule.java index 7af7863b2ce4e..101c6f49d02ff 100644 --- a/connect/basic-auth-extension/src/main/java/org/apache/kafka/connect/rest/basic/auth/extenstion/PropertyFileLoginModule.java +++ b/connect/basic-auth-extension/src/main/java/org/apache/kafka/connect/rest/basic/auth/extension/PropertyFileLoginModule.java @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.kafka.connect.rest.basic.auth.extenstion; +package org.apache.kafka.connect.rest.basic.auth.extension; import org.apache.kafka.common.config.ConfigException; import org.slf4j.Logger; diff --git a/connect/basic-auth-extension/src/main/resources/META-INF/services/org.apache.kafka.connect.rest.ConnectRestExtension b/connect/basic-auth-extension/src/main/resources/META-INF/services/org.apache.kafka.connect.rest.ConnectRestExtension index 098c9473d05c8..ba7ae5b580d80 100644 --- a/connect/basic-auth-extension/src/main/resources/META-INF/services/org.apache.kafka.connect.rest.ConnectRestExtension +++ b/connect/basic-auth-extension/src/main/resources/META-INF/services/org.apache.kafka.connect.rest.ConnectRestExtension @@ -13,4 +13,4 @@ # See the License for the specific language governing permissions and # limitations under the License. -org.apache.kafka.connect.rest.basic.auth.extenstion.BasicAuthSecurityRestExtension \ No newline at end of file +org.apache.kafka.connect.rest.basic.auth.extension.BasicAuthSecurityRestExtension \ No newline at end of file diff --git a/connect/basic-auth-extension/src/test/java/org/apache/kafka/connect/rest/basic/auth/extenstion/JaasBasicAuthFilterTest.java b/connect/basic-auth-extension/src/test/java/org/apache/kafka/connect/rest/basic/auth/extension/JaasBasicAuthFilterTest.java similarity index 79% rename from connect/basic-auth-extension/src/test/java/org/apache/kafka/connect/rest/basic/auth/extenstion/JaasBasicAuthFilterTest.java rename to connect/basic-auth-extension/src/test/java/org/apache/kafka/connect/rest/basic/auth/extension/JaasBasicAuthFilterTest.java index 80299f81c298b..c81f8f6fff6f4 100644 --- a/connect/basic-auth-extension/src/test/java/org/apache/kafka/connect/rest/basic/auth/extenstion/JaasBasicAuthFilterTest.java +++ b/connect/basic-auth-extension/src/test/java/org/apache/kafka/connect/rest/basic/auth/extension/JaasBasicAuthFilterTest.java @@ -15,14 +15,17 @@ * limitations under the License. */ -package org.apache.kafka.connect.rest.basic.auth.extenstion; +package org.apache.kafka.connect.rest.basic.auth.extension; +import javax.ws.rs.HttpMethod; +import javax.ws.rs.core.UriInfo; import org.apache.kafka.common.security.JaasUtils; import org.easymock.EasyMock; 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.MockStrict; import org.powermock.core.classloader.annotations.PowerMockIgnore; import org.powermock.modules.junit4.PowerMockRunner; @@ -52,6 +55,9 @@ public class JaasBasicAuthFilterTest { private String previousJaasConfig; private Configuration previousConfiguration; + @MockStrict + private UriInfo uriInfo; + @Before public void setup() throws IOException { EasyMock.reset(requestContext); @@ -137,7 +143,34 @@ public void testNoFileOption() throws IOException { jaasBasicAuthFilter.filter(requestContext); } + @Test + public void testPostWithoutAppropriateCredential() throws IOException { + EasyMock.expect(requestContext.getMethod()).andReturn(HttpMethod.POST); + EasyMock.expect(requestContext.getUriInfo()).andReturn(uriInfo); + EasyMock.expect(uriInfo.getPath()).andReturn("connectors/connName/tasks"); + + PowerMock.replayAll(); + jaasBasicAuthFilter.filter(requestContext); + EasyMock.verify(requestContext); + } + + @Test + public void testPostNotChangingConnectorTask() throws IOException { + EasyMock.expect(requestContext.getMethod()).andReturn(HttpMethod.POST); + EasyMock.expect(requestContext.getUriInfo()).andReturn(uriInfo); + EasyMock.expect(uriInfo.getPath()).andReturn("local:randomport/connectors/connName"); + String authHeader = "Basic" + Base64.getEncoder().encodeToString(("user" + ":" + "password").getBytes()); + EasyMock.expect(requestContext.getHeaderString(JaasBasicAuthFilter.AUTHORIZATION)) + .andReturn(authHeader); + requestContext.abortWith(EasyMock.anyObject(Response.class)); + EasyMock.expectLastCall(); + PowerMock.replayAll(); + jaasBasicAuthFilter.filter(requestContext); + EasyMock.verify(requestContext); + } + private void setMock(String authorization, String username, String password, boolean exceptionCase) { + EasyMock.expect(requestContext.getMethod()).andReturn(HttpMethod.GET); String authHeader = authorization + " " + Base64.getEncoder().encodeToString((username + ":" + password).getBytes()); EasyMock.expect(requestContext.getHeaderString(JaasBasicAuthFilter.AUTHORIZATION)) .andReturn(authHeader); @@ -152,10 +185,9 @@ private void setupJaasConfig(String loginModule, String credentialFilePath, bool File jaasConfigFile = File.createTempFile("ks-jaas-", ".conf"); jaasConfigFile.deleteOnExit(); previousJaasConfig = System.setProperty(JaasUtils.JAVA_LOGIN_CONFIG_PARAM, jaasConfigFile.getPath()); - List lines; lines = new ArrayList<>(); - lines.add(loginModule + " { org.apache.kafka.connect.rest.basic.auth.extenstion.PropertyFileLoginModule required "); + lines.add(loginModule + " { org.apache.kafka.connect.rest.basic.auth.extension.PropertyFileLoginModule required "); if (includeFileOptions) { lines.add("file=\"" + credentialFilePath + "\""); } diff --git a/connect/json/src/main/java/org/apache/kafka/connect/json/JsonConverter.java b/connect/json/src/main/java/org/apache/kafka/connect/json/JsonConverter.java index c1322b1ecdbe0..546fcf0daaa68 100644 --- a/connect/json/src/main/java/org/apache/kafka/connect/json/JsonConverter.java +++ b/connect/json/src/main/java/org/apache/kafka/connect/json/JsonConverter.java @@ -317,6 +317,10 @@ public SchemaAndValue toConnectHeader(String topic, String headerKey, byte[] val @Override public byte[] fromConnectData(String topic, Schema schema, Object value) { + if (schema == null && value == null) { + return null; + } + JsonNode jsonValue = enableSchemas ? convertToJsonWithEnvelope(schema, value) : convertToJsonWithoutEnvelope(schema, value); try { return serializer.serialize(topic, jsonValue); @@ -328,13 +332,19 @@ public byte[] fromConnectData(String topic, Schema schema, Object value) { @Override public SchemaAndValue toConnectData(String topic, byte[] value) { JsonNode jsonValue; + + // This handles a tombstone message + if (value == null) { + return SchemaAndValue.NULL; + } + try { jsonValue = deserializer.deserialize(topic, value); } catch (SerializationException e) { throw new DataException("Converting byte[] to Kafka Connect data failed due to serialization error: ", e); } - if (enableSchemas && (jsonValue == null || !jsonValue.isObject() || jsonValue.size() != 2 || !jsonValue.has("schema") || !jsonValue.has("payload"))) + if (enableSchemas && (!jsonValue.isObject() || jsonValue.size() != 2 || !jsonValue.has(JsonSchema.ENVELOPE_SCHEMA_FIELD_NAME) || !jsonValue.has(JsonSchema.ENVELOPE_PAYLOAD_FIELD_NAME))) throw new DataException("JsonConverter with schemas.enable requires \"schema\" and \"payload\" fields and may not contain additional fields." + " If you are trying to deserialize plain JSON data, set schemas.enable=false in your converter configuration."); @@ -342,23 +352,16 @@ public SchemaAndValue toConnectData(String topic, byte[] value) { // 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); + envelope.set(JsonSchema.ENVELOPE_SCHEMA_FIELD_NAME, null); + envelope.set(JsonSchema.ENVELOPE_PAYLOAD_FIELD_NAME, jsonValue); jsonValue = envelope; } - return jsonToConnect(jsonValue); - } - - private SchemaAndValue jsonToConnect(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 Kafka Connect must be in envelope containing schema"); - Schema schema = asConnectSchema(jsonValue.get(JsonSchema.ENVELOPE_SCHEMA_FIELD_NAME)); - return new SchemaAndValue(schema, convertToConnect(schema, jsonValue.get(JsonSchema.ENVELOPE_PAYLOAD_FIELD_NAME))); + return new SchemaAndValue( + schema, + convertToConnect(schema, jsonValue.get(JsonSchema.ENVELOPE_PAYLOAD_FIELD_NAME)) + ); } public ObjectNode asJsonSchema(Schema schema) { diff --git a/connect/json/src/test/java/org/apache/kafka/connect/json/JsonConverterTest.java b/connect/json/src/test/java/org/apache/kafka/connect/json/JsonConverterTest.java index 7686fdb7ab524..d5bb24c3534e9 100644 --- a/connect/json/src/test/java/org/apache/kafka/connect/json/JsonConverterTest.java +++ b/connect/json/src/test/java/org/apache/kafka/connect/json/JsonConverterTest.java @@ -172,10 +172,13 @@ public void structToConnect() { assertEquals(new SchemaAndValue(expectedSchema, expected), converted); } - @Test(expected = DataException.class) + @Test public void nullToConnect() { - // When schemas are enabled, trying to decode a null should be an error -- we should *always* have the envelope - assertEquals(SchemaAndValue.NULL, converter.toConnectData(TOPIC, null)); + // When schemas are enabled, trying to decode a tombstone should be an empty envelope + // the behavior is the same as when the json is "{ "schema": null, "payload": null }" + // to keep compatibility with the record + SchemaAndValue converted = converter.toConnectData(TOPIC, null); + assertEquals(SchemaAndValue.NULL, converted); } @Test @@ -696,6 +699,23 @@ public void nullSchemaAndMapNonStringKeysToJson() { ); } + @Test + public void nullSchemaAndNullValueToJson() { + // This characterizes the production of tombstone messages when Json schemas is enabled + Map props = Collections.singletonMap("schemas.enable", true); + converter.configure(props, true); + byte[] converted = converter.fromConnectData(TOPIC, null, null); + assertNull(converted); + } + + @Test + public void nullValueToJson() { + // This characterizes the production of tombstone messages when Json schemas is not enabled + Map props = Collections.singletonMap("schemas.enable", false); + converter.configure(props, true); + byte[] converted = converter.fromConnectData(TOPIC, null, null); + assertNull(converted); + } @Test(expected = DataException.class) public void mismatchSchemaJson() { @@ -703,8 +723,6 @@ public void mismatchSchemaJson() { converter.fromConnectData(TOPIC, Schema.FLOAT64_SCHEMA, true); } - - @Test public void noSchemaToConnect() { Map props = Collections.singletonMap("schemas.enable", false); diff --git a/connect/runtime/src/main/java/org/apache/kafka/connect/cli/ConnectDistributed.java b/connect/runtime/src/main/java/org/apache/kafka/connect/cli/ConnectDistributed.java index f8c15de8ef4a9..17d65ac678d45 100644 --- a/connect/runtime/src/main/java/org/apache/kafka/connect/cli/ConnectDistributed.java +++ b/connect/runtime/src/main/java/org/apache/kafka/connect/cli/ConnectDistributed.java @@ -38,6 +38,7 @@ import org.slf4j.LoggerFactory; import java.net.URI; +import java.util.Arrays; import java.util.Collections; import java.util.Map; @@ -53,62 +54,26 @@ public class ConnectDistributed { private static final Logger log = LoggerFactory.getLogger(ConnectDistributed.class); - public static void main(String[] args) throws Exception { - if (args.length < 1) { + private final Time time = Time.SYSTEM; + private final long initStart = time.hiResClockMs(); + + public static void main(String[] args) { + + if (args.length < 1 || Arrays.asList(args).contains("--help")) { log.info("Usage: ConnectDistributed worker.properties"); Exit.exit(1); } try { - Time time = Time.SYSTEM; - log.info("Kafka Connect distributed worker initializing ..."); - long initStart = time.hiResClockMs(); WorkerInfo initInfo = new WorkerInfo(); initInfo.logAll(); String workerPropsFile = args[0]; Map workerProps = !workerPropsFile.isEmpty() ? - Utils.propsToStringMap(Utils.loadProps(workerPropsFile)) : Collections.emptyMap(); - - log.info("Scanning for plugin classes. This might take a moment ..."); - Plugins plugins = new Plugins(workerProps); - plugins.compareAndSwapWithDelegatingLoader(); - DistributedConfig config = new DistributedConfig(workerProps); - - String kafkaClusterId = ConnectUtils.lookupKafkaClusterId(config); - log.debug("Kafka cluster ID: {}", kafkaClusterId); - - RestServer rest = new RestServer(config); - URI advertisedUrl = rest.advertisedUrl(); - String workerId = advertisedUrl.getHost() + ":" + advertisedUrl.getPort(); - - KafkaOffsetBackingStore offsetBackingStore = new KafkaOffsetBackingStore(); - offsetBackingStore.configure(config); - - Worker worker = new Worker(workerId, time, plugins, config, offsetBackingStore); - WorkerConfigTransformer configTransformer = worker.configTransformer(); - - Converter internalValueConverter = worker.getInternalValueConverter(); - StatusBackingStore statusBackingStore = new KafkaStatusBackingStore(time, internalValueConverter); - statusBackingStore.configure(config); - - ConfigBackingStore configBackingStore = new KafkaConfigBackingStore( - internalValueConverter, - config, - configTransformer); - - DistributedHerder herder = new DistributedHerder(config, time, worker, - kafkaClusterId, statusBackingStore, configBackingStore, - advertisedUrl.toString()); - final Connect connect = new Connect(herder, rest); - log.info("Kafka Connect distributed worker initialization took {}ms", time.hiResClockMs() - initStart); - try { - connect.start(); - } catch (Exception e) { - log.error("Failed to start Connect", e); - connect.stop(); - Exit.exit(3); - } + Utils.propsToStringMap(Utils.loadProps(workerPropsFile)) : Collections.emptyMap(); + + ConnectDistributed connectDistributed = new ConnectDistributed(); + Connect connect = connectDistributed.startConnect(workerProps); // Shutdown will be triggered by Ctrl-C or via HTTP shutdown request connect.awaitStop(); @@ -118,4 +83,52 @@ public static void main(String[] args) throws Exception { Exit.exit(2); } } + + public Connect startConnect(Map workerProps) { + log.info("Scanning for plugin classes. This might take a moment ..."); + Plugins plugins = new Plugins(workerProps); + plugins.compareAndSwapWithDelegatingLoader(); + DistributedConfig config = new DistributedConfig(workerProps); + + String kafkaClusterId = ConnectUtils.lookupKafkaClusterId(config); + log.debug("Kafka cluster ID: {}", kafkaClusterId); + + RestServer rest = new RestServer(config); + rest.initializeServer(); + + URI advertisedUrl = rest.advertisedUrl(); + String workerId = advertisedUrl.getHost() + ":" + advertisedUrl.getPort(); + + KafkaOffsetBackingStore offsetBackingStore = new KafkaOffsetBackingStore(); + offsetBackingStore.configure(config); + + Worker worker = new Worker(workerId, time, plugins, config, offsetBackingStore); + WorkerConfigTransformer configTransformer = worker.configTransformer(); + + Converter internalValueConverter = worker.getInternalValueConverter(); + StatusBackingStore statusBackingStore = new KafkaStatusBackingStore(time, internalValueConverter); + statusBackingStore.configure(config); + + ConfigBackingStore configBackingStore = new KafkaConfigBackingStore( + internalValueConverter, + config, + configTransformer); + + DistributedHerder herder = new DistributedHerder(config, time, worker, + kafkaClusterId, statusBackingStore, configBackingStore, + advertisedUrl.toString()); + + final Connect connect = new Connect(herder, rest); + log.info("Kafka Connect distributed worker initialization took {}ms", time.hiResClockMs() - initStart); + try { + connect.start(); + } catch (Exception e) { + log.error("Failed to start Connect", e); + connect.stop(); + Exit.exit(3); + } + + return connect; + } + } diff --git a/connect/runtime/src/main/java/org/apache/kafka/connect/cli/ConnectStandalone.java b/connect/runtime/src/main/java/org/apache/kafka/connect/cli/ConnectStandalone.java index aba9d9c32aa4b..1ebfd3a002dba 100644 --- a/connect/runtime/src/main/java/org/apache/kafka/connect/cli/ConnectStandalone.java +++ b/connect/runtime/src/main/java/org/apache/kafka/connect/cli/ConnectStandalone.java @@ -82,6 +82,8 @@ public static void main(String[] args) throws Exception { log.debug("Kafka cluster ID: {}", kafkaClusterId); RestServer rest = new RestServer(config); + rest.initializeServer(); + URI advertisedUrl = rest.advertisedUrl(); String workerId = advertisedUrl.getHost() + ":" + advertisedUrl.getPort(); diff --git a/connect/api/src/main/java/org/apache/kafka/connect/storage/DoubleConverter.java b/connect/runtime/src/main/java/org/apache/kafka/connect/converters/DoubleConverter.java similarity index 91% rename from connect/api/src/main/java/org/apache/kafka/connect/storage/DoubleConverter.java rename to connect/runtime/src/main/java/org/apache/kafka/connect/converters/DoubleConverter.java index 04019a7a52900..684caa1c658e3 100644 --- a/connect/api/src/main/java/org/apache/kafka/connect/storage/DoubleConverter.java +++ b/connect/runtime/src/main/java/org/apache/kafka/connect/converters/DoubleConverter.java @@ -14,11 +14,13 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.kafka.connect.storage; +package org.apache.kafka.connect.converters; import org.apache.kafka.common.serialization.DoubleDeserializer; import org.apache.kafka.common.serialization.DoubleSerializer; import org.apache.kafka.connect.data.Schema; +import org.apache.kafka.connect.storage.Converter; +import org.apache.kafka.connect.storage.HeaderConverter; /** * {@link Converter} and {@link HeaderConverter} implementation that only supports serializing to and deserializing from double values. diff --git a/connect/api/src/main/java/org/apache/kafka/connect/storage/FloatConverter.java b/connect/runtime/src/main/java/org/apache/kafka/connect/converters/FloatConverter.java similarity index 91% rename from connect/api/src/main/java/org/apache/kafka/connect/storage/FloatConverter.java rename to connect/runtime/src/main/java/org/apache/kafka/connect/converters/FloatConverter.java index 16bf0e0f93f65..3f92b965cec4d 100644 --- a/connect/api/src/main/java/org/apache/kafka/connect/storage/FloatConverter.java +++ b/connect/runtime/src/main/java/org/apache/kafka/connect/converters/FloatConverter.java @@ -14,11 +14,13 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.kafka.connect.storage; +package org.apache.kafka.connect.converters; import org.apache.kafka.common.serialization.FloatDeserializer; import org.apache.kafka.common.serialization.FloatSerializer; import org.apache.kafka.connect.data.Schema; +import org.apache.kafka.connect.storage.Converter; +import org.apache.kafka.connect.storage.HeaderConverter; /** * {@link Converter} and {@link HeaderConverter} implementation that only supports serializing to and deserializing from float values. diff --git a/connect/api/src/main/java/org/apache/kafka/connect/storage/IntegerConverter.java b/connect/runtime/src/main/java/org/apache/kafka/connect/converters/IntegerConverter.java similarity index 91% rename from connect/api/src/main/java/org/apache/kafka/connect/storage/IntegerConverter.java rename to connect/runtime/src/main/java/org/apache/kafka/connect/converters/IntegerConverter.java index 6f3c78a0a73dd..f5388ce35bb69 100644 --- a/connect/api/src/main/java/org/apache/kafka/connect/storage/IntegerConverter.java +++ b/connect/runtime/src/main/java/org/apache/kafka/connect/converters/IntegerConverter.java @@ -14,11 +14,13 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.kafka.connect.storage; +package org.apache.kafka.connect.converters; import org.apache.kafka.common.serialization.IntegerDeserializer; import org.apache.kafka.common.serialization.IntegerSerializer; import org.apache.kafka.connect.data.Schema; +import org.apache.kafka.connect.storage.Converter; +import org.apache.kafka.connect.storage.HeaderConverter; /** * {@link Converter} and {@link HeaderConverter} implementation that only supports serializing to and deserializing from integer values. diff --git a/connect/api/src/main/java/org/apache/kafka/connect/storage/LongConverter.java b/connect/runtime/src/main/java/org/apache/kafka/connect/converters/LongConverter.java similarity index 91% rename from connect/api/src/main/java/org/apache/kafka/connect/storage/LongConverter.java rename to connect/runtime/src/main/java/org/apache/kafka/connect/converters/LongConverter.java index 600c304250282..f91f4fad96392 100644 --- a/connect/api/src/main/java/org/apache/kafka/connect/storage/LongConverter.java +++ b/connect/runtime/src/main/java/org/apache/kafka/connect/converters/LongConverter.java @@ -14,11 +14,13 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.kafka.connect.storage; +package org.apache.kafka.connect.converters; import org.apache.kafka.common.serialization.LongDeserializer; import org.apache.kafka.common.serialization.LongSerializer; import org.apache.kafka.connect.data.Schema; +import org.apache.kafka.connect.storage.Converter; +import org.apache.kafka.connect.storage.HeaderConverter; /** * {@link Converter} and {@link HeaderConverter} implementation that only supports serializing to and deserializing from long values. diff --git a/connect/api/src/main/java/org/apache/kafka/connect/storage/NumberConverter.java b/connect/runtime/src/main/java/org/apache/kafka/connect/converters/NumberConverter.java similarity index 94% rename from connect/api/src/main/java/org/apache/kafka/connect/storage/NumberConverter.java rename to connect/runtime/src/main/java/org/apache/kafka/connect/converters/NumberConverter.java index 9180444c0c8b6..131a0975bfa25 100644 --- a/connect/api/src/main/java/org/apache/kafka/connect/storage/NumberConverter.java +++ b/connect/runtime/src/main/java/org/apache/kafka/connect/converters/NumberConverter.java @@ -14,7 +14,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.kafka.connect.storage; +package org.apache.kafka.connect.converters; import org.apache.kafka.common.config.ConfigDef; import org.apache.kafka.common.errors.SerializationException; @@ -23,6 +23,10 @@ import org.apache.kafka.connect.data.Schema; import org.apache.kafka.connect.data.SchemaAndValue; import org.apache.kafka.connect.errors.DataException; +import org.apache.kafka.connect.storage.Converter; +import org.apache.kafka.connect.storage.ConverterType; +import org.apache.kafka.connect.storage.HeaderConverter; +import org.apache.kafka.connect.storage.StringConverterConfig; import java.io.IOException; import java.util.HashMap; diff --git a/connect/api/src/main/java/org/apache/kafka/connect/storage/NumberConverterConfig.java b/connect/runtime/src/main/java/org/apache/kafka/connect/converters/NumberConverterConfig.java similarity index 93% rename from connect/api/src/main/java/org/apache/kafka/connect/storage/NumberConverterConfig.java rename to connect/runtime/src/main/java/org/apache/kafka/connect/converters/NumberConverterConfig.java index 2f7019d006933..49ad98673621d 100644 --- a/connect/api/src/main/java/org/apache/kafka/connect/storage/NumberConverterConfig.java +++ b/connect/runtime/src/main/java/org/apache/kafka/connect/converters/NumberConverterConfig.java @@ -14,9 +14,10 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.kafka.connect.storage; +package org.apache.kafka.connect.converters; import org.apache.kafka.common.config.ConfigDef; +import org.apache.kafka.connect.storage.ConverterConfig; import java.util.Map; diff --git a/connect/api/src/main/java/org/apache/kafka/connect/storage/ShortConverter.java b/connect/runtime/src/main/java/org/apache/kafka/connect/converters/ShortConverter.java similarity index 91% rename from connect/api/src/main/java/org/apache/kafka/connect/storage/ShortConverter.java rename to connect/runtime/src/main/java/org/apache/kafka/connect/converters/ShortConverter.java index 9a769ffc1a529..1c455b189a3f9 100644 --- a/connect/api/src/main/java/org/apache/kafka/connect/storage/ShortConverter.java +++ b/connect/runtime/src/main/java/org/apache/kafka/connect/converters/ShortConverter.java @@ -14,11 +14,13 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.kafka.connect.storage; +package org.apache.kafka.connect.converters; import org.apache.kafka.common.serialization.ShortDeserializer; import org.apache.kafka.common.serialization.ShortSerializer; import org.apache.kafka.connect.data.Schema; +import org.apache.kafka.connect.storage.Converter; +import org.apache.kafka.connect.storage.HeaderConverter; /** * {@link Converter} and {@link HeaderConverter} implementation that only supports serializing to and deserializing from short values. diff --git a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/AbstractHerder.java b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/AbstractHerder.java index b5e0ec2c07b26..106b788726f83 100644 --- a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/AbstractHerder.java +++ b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/AbstractHerder.java @@ -20,9 +20,11 @@ import org.apache.kafka.common.config.ConfigDef; import org.apache.kafka.common.config.ConfigDef.ConfigKey; import org.apache.kafka.common.config.ConfigDef.Type; +import org.apache.kafka.common.config.ConfigTransformer; import org.apache.kafka.common.config.ConfigValue; import org.apache.kafka.connect.connector.Connector; import org.apache.kafka.connect.errors.NotFoundException; +import org.apache.kafka.connect.runtime.distributed.ClusterConfigState; import org.apache.kafka.connect.runtime.isolation.Plugins; import org.apache.kafka.connect.runtime.rest.entities.ConfigInfo; import org.apache.kafka.connect.runtime.rest.entities.ConfigInfos; @@ -46,6 +48,7 @@ import java.util.Collection; import java.util.Collections; import java.util.HashMap; +import java.util.HashSet; import java.util.LinkedHashMap; import java.util.LinkedHashSet; import java.util.LinkedList; @@ -53,6 +56,8 @@ import java.util.Map; import java.util.Set; import java.util.concurrent.ConcurrentHashMap; +import java.util.regex.Matcher; +import java.util.regex.Pattern; /** * Abstract Herder implementation which handles connector/task lifecycle tracking. Extensions @@ -246,6 +251,9 @@ protected Map validateBasicConnectorConfig(Connector connec @Override public ConfigInfos validateConnectorConfig(Map connectorProps) { + if (worker.configTransformer() != null) { + connectorProps = worker.configTransformer().transform(connectorProps); + } String connType = connectorProps.get(ConnectorConfig.CONNECTOR_CLASS_CONFIG); if (connType == null) throw new BadRequestException("Connector config " + connectorProps + " contains no connector type"); @@ -428,4 +436,43 @@ private String trace(Throwable t) { return null; } } + + /* + * Performs a reverse transformation on a set of task configs, by replacing values with variable references. + */ + public static List> reverseTransform(String connName, + ClusterConfigState configState, + List> configs) { + + // Find the config keys in the raw connector config that have variable references + Map rawConnConfig = configState.rawConnectorConfig(connName); + Set connKeysWithVariableValues = keysWithVariableValues(rawConnConfig, ConfigTransformer.DEFAULT_PATTERN); + + List> result = new ArrayList<>(); + for (Map config : configs) { + Map newConfig = new HashMap<>(config); + for (String key : connKeysWithVariableValues) { + if (newConfig.containsKey(key)) { + newConfig.put(key, rawConnConfig.get(key)); + } + } + result.add(newConfig); + } + return result; + } + + // Visible for testing + static Set keysWithVariableValues(Map rawConfig, Pattern pattern) { + Set keys = new HashSet<>(); + for (Map.Entry config : rawConfig.entrySet()) { + if (config.getValue() != null) { + Matcher matcher = pattern.matcher(config.getValue()); + if (matcher.find()) { + keys.add(config.getKey()); + } + } + } + return keys; + } + } diff --git a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/Connect.java b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/Connect.java index 846ed1a883570..4a0bcabea8982 100644 --- a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/Connect.java +++ b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/Connect.java @@ -20,6 +20,7 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import java.net.URI; import java.util.concurrent.CountDownLatch; import java.util.concurrent.atomic.AtomicBoolean; @@ -50,7 +51,7 @@ public void start() { Runtime.getRuntime().addShutdownHook(shutdownHook); herder.start(); - rest.start(herder); + rest.initializeResources(herder); log.info("Kafka Connect started"); } finally { @@ -82,6 +83,11 @@ public void awaitStop() { } } + // Visible for testing + public URI restUrl() { + return rest.serverUrl(); + } + private class ShutdownHook extends Thread { @Override public void run() { diff --git a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/ConnectorConfig.java b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/ConnectorConfig.java index f98469e5b5ab8..d030fed873a88 100644 --- a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/ConnectorConfig.java +++ b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/ConnectorConfig.java @@ -35,6 +35,7 @@ import java.util.HashSet; import java.util.LinkedHashSet; import java.util.List; +import java.util.Locale; import java.util.Map; import static org.apache.kafka.common.config.ConfigDef.NonEmptyStringWithoutControlChars.nonEmptyStringWithoutControlChars; @@ -105,8 +106,8 @@ public class ConnectorConfig extends AbstractConfig { "indicates that a configuration value will expire in the future."; private static final String CONFIG_RELOAD_ACTION_DISPLAY = "Reload Action"; - public static final String CONFIG_RELOAD_ACTION_NONE = Herder.ConfigReloadAction.NONE.toString(); - public static final String CONFIG_RELOAD_ACTION_RESTART = Herder.ConfigReloadAction.RESTART.toString(); + public static final String CONFIG_RELOAD_ACTION_NONE = Herder.ConfigReloadAction.NONE.name().toLowerCase(Locale.ROOT); + public static final String CONFIG_RELOAD_ACTION_RESTART = Herder.ConfigReloadAction.RESTART.name().toLowerCase(Locale.ROOT); public static final String ERRORS_RETRY_TIMEOUT_CONFIG = "errors.retry.timeout"; public static final String ERRORS_RETRY_TIMEOUT_DISPLAY = "Retry Timeout for Errors"; @@ -120,7 +121,7 @@ public class ConnectorConfig extends AbstractConfig { public static final String ERRORS_RETRY_MAX_DELAY_DOC = "The maximum duration in milliseconds between consecutive retry attempts. " + "Jitter will be added to the delay once this limit is reached to prevent thundering herd issues."; - public static final String ERRORS_TOLERANCE_CONFIG = "errors.allowed.max"; + public static final String ERRORS_TOLERANCE_CONFIG = "errors.tolerance"; public static final String ERRORS_TOLERANCE_DISPLAY = "Error Tolerance"; public static final ToleranceType ERRORS_TOLERANCE_DEFAULT = ToleranceType.NONE; public static final String ERRORS_TOLERANCE_DOC = "Behavior for tolerating errors during connector operation. 'none' is the default value " + diff --git a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/Herder.java b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/Herder.java index 5c7cc1429aaf0..c572e20b52f2b 100644 --- a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/Herder.java +++ b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/Herder.java @@ -148,12 +148,6 @@ public interface Herder { */ void restartTask(ConnectorTaskId id, Callback cb); - /** - * Get the configuration reload action. - * @param connName name of the connector - */ - ConfigReloadAction connectorConfigReloadAction(final String connName); - /** * Restart the connector. * @param connName name of the connector diff --git a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/SinkConnectorConfig.java b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/SinkConnectorConfig.java index 9629f8f0e42e1..d9d140b9cdcd9 100644 --- a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/SinkConnectorConfig.java +++ b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/SinkConnectorConfig.java @@ -52,10 +52,24 @@ public class SinkConnectorConfig extends ConnectorConfig { public static final String DLQ_TOPIC_DEFAULT = ""; private static final String DLQ_TOPIC_DISPLAY = "Dead Letter Queue Topic Name"; + public static final String DLQ_TOPIC_REPLICATION_FACTOR_CONFIG = DLQ_PREFIX + "topic.replication.factor"; + private static final String DLQ_TOPIC_REPLICATION_FACTOR_CONFIG_DOC = "Replication factor used to create the dead letter queue topic when it doesn't already exist."; + public static final short DLQ_TOPIC_REPLICATION_FACTOR_CONFIG_DEFAULT = 3; + private static final String DLQ_TOPIC_REPLICATION_FACTOR_CONFIG_DISPLAY = "Dead Letter Queue Topic Replication Factor"; + + public static final String DLQ_CONTEXT_HEADERS_ENABLE_CONFIG = DLQ_PREFIX + "context.headers.enable"; + public static final boolean DLQ_CONTEXT_HEADERS_ENABLE_DEFAULT = false; + public static final String DLQ_CONTEXT_HEADERS_ENABLE_DOC = "If true, add headers containing error context to the messages " + + "written to the dead letter queue. To avoid clashing with headers from the original record, all error context header " + + "keys, all error context header keys will start with __connect.errors."; + private static final String DLQ_CONTEXT_HEADERS_ENABLE_DISPLAY = "Enable Error Context Headers"; + static ConfigDef config = ConnectorConfig.configDef() .define(TOPICS_CONFIG, ConfigDef.Type.LIST, TOPICS_DEFAULT, ConfigDef.Importance.HIGH, TOPICS_DOC, COMMON_GROUP, 4, ConfigDef.Width.LONG, TOPICS_DISPLAY) .define(TOPICS_REGEX_CONFIG, ConfigDef.Type.STRING, TOPICS_REGEX_DEFAULT, new RegexValidator(), ConfigDef.Importance.HIGH, TOPICS_REGEX_DOC, COMMON_GROUP, 4, ConfigDef.Width.LONG, TOPICS_REGEX_DISPLAY) - .define(DLQ_TOPIC_NAME_CONFIG, ConfigDef.Type.STRING, DLQ_TOPIC_DEFAULT, Importance.MEDIUM, DLQ_TOPIC_NAME_DOC, ERROR_GROUP, 6, ConfigDef.Width.MEDIUM, DLQ_TOPIC_DISPLAY); + .define(DLQ_TOPIC_NAME_CONFIG, ConfigDef.Type.STRING, DLQ_TOPIC_DEFAULT, Importance.MEDIUM, DLQ_TOPIC_NAME_DOC, ERROR_GROUP, 6, ConfigDef.Width.MEDIUM, DLQ_TOPIC_DISPLAY) + .define(DLQ_TOPIC_REPLICATION_FACTOR_CONFIG, ConfigDef.Type.SHORT, DLQ_TOPIC_REPLICATION_FACTOR_CONFIG_DEFAULT, Importance.MEDIUM, DLQ_TOPIC_REPLICATION_FACTOR_CONFIG_DOC, ERROR_GROUP, 7, ConfigDef.Width.MEDIUM, DLQ_TOPIC_REPLICATION_FACTOR_CONFIG_DISPLAY) + .define(DLQ_CONTEXT_HEADERS_ENABLE_CONFIG, ConfigDef.Type.BOOLEAN, DLQ_CONTEXT_HEADERS_ENABLE_DEFAULT, Importance.MEDIUM, DLQ_CONTEXT_HEADERS_ENABLE_DOC, ERROR_GROUP, 8, ConfigDef.Width.MEDIUM, DLQ_CONTEXT_HEADERS_ENABLE_DISPLAY); public static ConfigDef configDef() { return config; @@ -97,4 +111,12 @@ public static boolean hasTopicsRegexConfig(Map props) { public String dlqTopicName() { return getString(DLQ_TOPIC_NAME_CONFIG); } + + public short dlqTopicReplicationFactor() { + return getShort(DLQ_TOPIC_REPLICATION_FACTOR_CONFIG); + } + + public boolean isDlqContextHeadersEnabled() { + return getBoolean(DLQ_CONTEXT_HEADERS_ENABLE_CONFIG); + } } diff --git a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/TransformationChain.java b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/TransformationChain.java index 3680905be2a36..a077a01af668a 100644 --- a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/TransformationChain.java +++ b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/TransformationChain.java @@ -20,11 +20,15 @@ import org.apache.kafka.connect.runtime.errors.RetryWithToleranceOperator; import org.apache.kafka.connect.runtime.errors.Stage; import org.apache.kafka.connect.transforms.Transformation; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; import java.util.List; import java.util.Objects; +import java.util.StringJoiner; public class TransformationChain> { + private static final Logger log = LoggerFactory.getLogger(TransformationChain.class); private final List> transformations; private final RetryWithToleranceOperator retryWithToleranceOperator; @@ -40,6 +44,8 @@ public R apply(R record) { for (final Transformation transformation : transformations) { final R current = record; + log.trace("Applying transformation {} to {}", + transformation.getClass().getName(), record); // execute the operation record = retryWithToleranceOperator.execute(() -> transformation.apply(current), Stage.TRANSFORMATION, transformation.getClass()); @@ -68,4 +74,11 @@ public int hashCode() { return Objects.hash(transformations); } + public String toString() { + StringJoiner chain = new StringJoiner(", ", getClass().getName() + "{", "}"); + for (Transformation transformation : transformations) { + chain.add(transformation.getClass().getName()); + } + return chain.toString(); + } } diff --git a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/Worker.java b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/Worker.java index 97e68faa4ca9f..9266c4f38646e 100644 --- a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/Worker.java +++ b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/Worker.java @@ -19,7 +19,7 @@ import org.apache.kafka.clients.producer.KafkaProducer; import org.apache.kafka.clients.producer.ProducerConfig; import org.apache.kafka.common.MetricName; -import org.apache.kafka.common.config.ConfigProvider; +import org.apache.kafka.common.config.provider.ConfigProvider; import org.apache.kafka.common.metrics.Sensor; import org.apache.kafka.common.metrics.stats.Frequencies; import org.apache.kafka.common.metrics.stats.Total; @@ -43,10 +43,10 @@ import org.apache.kafka.connect.sink.SinkTask; import org.apache.kafka.connect.source.SourceRecord; import org.apache.kafka.connect.source.SourceTask; +import org.apache.kafka.connect.storage.CloseableOffsetStorageReader; import org.apache.kafka.connect.storage.Converter; import org.apache.kafka.connect.storage.HeaderConverter; import org.apache.kafka.connect.storage.OffsetBackingStore; -import org.apache.kafka.connect.storage.OffsetStorageReader; import org.apache.kafka.connect.storage.OffsetStorageReaderImpl; import org.apache.kafka.connect.storage.OffsetStorageWriter; import org.apache.kafka.connect.util.ConnectorTaskId; @@ -102,9 +102,21 @@ public Worker( Plugins plugins, WorkerConfig config, OffsetBackingStore offsetBackingStore + ) { + this(workerId, time, plugins, config, offsetBackingStore, Executors.newCachedThreadPool()); + } + + @SuppressWarnings("deprecation") + Worker( + String workerId, + Time time, + Plugins plugins, + WorkerConfig config, + OffsetBackingStore offsetBackingStore, + ExecutorService executorService ) { this.metrics = new ConnectMetrics(workerId, config, time); - this.executor = Executors.newCachedThreadPool(); + this.executor = executorService; this.workerId = workerId; this.time = time; this.plugins = plugins; @@ -401,10 +413,10 @@ public boolean startTask( final WorkerTask workerTask; ClassLoader savedLoader = plugins.currentThreadLoader(); try { - final ConnectorConfig connConfig = new ConnectorConfig(plugins, connProps); - String connType = connConfig.getString(ConnectorConfig.CONNECTOR_CLASS_CONFIG); + String connType = connProps.get(ConnectorConfig.CONNECTOR_CLASS_CONFIG); ClassLoader connectorLoader = plugins.delegatingLoader().connectorLoader(connType); savedLoader = Plugins.compareAndSwapLoaders(connectorLoader); + final ConnectorConfig connConfig = new ConnectorConfig(plugins, connProps); final TaskConfig taskConfig = new TaskConfig(taskProps); final Class taskClass = taskConfig.getClass(TaskConfig.TASK_CLASS_CONFIG).asSubclass(Task.class); final Task task = plugins.newTask(taskClass); @@ -493,7 +505,8 @@ private WorkerTask buildWorkerTask(ClusterConfigState configState, if (task instanceof SourceTask) { retryWithToleranceOperator.reporters(sourceTaskReporters(id, connConfig, errorHandlingMetrics)); TransformationChain transformationChain = new TransformationChain<>(connConfig.transformations(), retryWithToleranceOperator); - OffsetStorageReader offsetReader = new OffsetStorageReaderImpl(offsetBackingStore, id.connector(), + log.info("Initializing: {}", transformationChain); + CloseableOffsetStorageReader offsetReader = new OffsetStorageReaderImpl(offsetBackingStore, id.connector(), internalKeyConverter, internalValueConverter); OffsetStorageWriter offsetWriter = new OffsetStorageWriter(offsetBackingStore, id.connector(), internalKeyConverter, internalValueConverter); @@ -505,6 +518,7 @@ private WorkerTask buildWorkerTask(ClusterConfigState configState, time, retryWithToleranceOperator); } else if (task instanceof SinkTask) { TransformationChain transformationChain = new TransformationChain<>(connConfig.transformations(), retryWithToleranceOperator); + log.info("Initializing: {}", transformationChain); SinkConnectorConfig sinkConfig = new SinkConnectorConfig(plugins, connConfig.originalsStrings()); retryWithToleranceOperator.reporters(sinkTaskReporters(id, sinkConfig, errorHandlingMetrics)); return new WorkerSinkTask(id, (SinkTask) task, statusListener, initialState, config, configState, metrics, keyConverter, @@ -523,14 +537,13 @@ ErrorHandlingMetrics errorHandlingMetrics(ConnectorTaskId id) { private List sinkTaskReporters(ConnectorTaskId id, SinkConnectorConfig connConfig, ErrorHandlingMetrics errorHandlingMetrics) { ArrayList reporters = new ArrayList<>(); - LogReporter logReporter = new LogReporter(id, connConfig); - logReporter.metrics(errorHandlingMetrics); + LogReporter logReporter = new LogReporter(id, connConfig, errorHandlingMetrics); reporters.add(logReporter); // check if topic for dead letter queue exists String topic = connConfig.dlqTopicName(); if (topic != null && !topic.isEmpty()) { - DeadLetterQueueReporter reporter = DeadLetterQueueReporter.createAndSetup(config, connConfig, producerProps); + DeadLetterQueueReporter reporter = DeadLetterQueueReporter.createAndSetup(config, id, connConfig, producerProps, errorHandlingMetrics); reporters.add(reporter); } @@ -540,8 +553,7 @@ private List sinkTaskReporters(ConnectorTaskId id, SinkConnectorC private List sourceTaskReporters(ConnectorTaskId id, ConnectorConfig connConfig, ErrorHandlingMetrics errorHandlingMetrics) { List reporters = new ArrayList<>(); - LogReporter logReporter = new LogReporter(id, connConfig); - logReporter.metrics(errorHandlingMetrics); + LogReporter logReporter = new LogReporter(id, connConfig, errorHandlingMetrics); reporters.add(logReporter); return reporters; diff --git a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/WorkerConfig.java b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/WorkerConfig.java index 355cfbb615b47..c703bfc4b7479 100644 --- a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/WorkerConfig.java +++ b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/WorkerConfig.java @@ -192,16 +192,18 @@ public class WorkerConfig extends AbstractConfig { + "/opt/connectors"; public static final String CONFIG_PROVIDERS_CONFIG = "config.providers"; - protected static final String CONFIG_PROVIDERS_DOC = "List of configuration providers. " - + "This is a comma-separated list of the fully-qualified names of the ConfigProvider implementations, " - + "in the order they will be created, configured, and used."; + protected static final String CONFIG_PROVIDERS_DOC = + "Comma-separated names of ConfigProvider classes, loaded and used " + + "in the order specified. Implementing the interface " + + "ConfigProvider allows you to replace variable references in connector configurations, " + + "such as for externalized secrets. "; public static final String REST_EXTENSION_CLASSES_CONFIG = "rest.extension.classes"; protected static final String REST_EXTENSION_CLASSES_DOC = "Comma-separated names of ConnectRestExtension classes, loaded and called " + "in the order specified. Implementing the interface " - + "ConnectRestExtension allows you to inject into Connect's REST API user defined resources like filters. " - + "Typically used to add custom capability like logging, security, etc."; + + "ConnectRestExtension allows you to inject into Connect's REST API user defined resources like filters. " + + "Typically used to add custom capability like logging, security, etc. "; public static final String METRICS_SAMPLE_WINDOW_MS_CONFIG = CommonClientConfigs.METRICS_SAMPLE_WINDOW_MS_CONFIG; public static final String METRICS_NUM_SAMPLES_CONFIG = CommonClientConfigs.METRICS_NUM_SAMPLES_CONFIG; @@ -327,6 +329,10 @@ private void logDeprecatedProperty(String propName, String propValue, String def } } + public Integer getRebalanceTimeout() { + return null; + } + @Override protected Map postProcessParsedConfig(final Map parsedValues) { return CommonClientConfigs.postProcessReconnectBackoffConfigs(this, parsedValues); diff --git a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/WorkerConfigTransformer.java b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/WorkerConfigTransformer.java index d91411cb8e869..3373d5ce32886 100644 --- a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/WorkerConfigTransformer.java +++ b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/WorkerConfigTransformer.java @@ -16,10 +16,15 @@ */ package org.apache.kafka.connect.runtime; -import org.apache.kafka.common.config.ConfigProvider; +import org.apache.kafka.common.config.ConfigDef; +import org.apache.kafka.common.config.provider.ConfigProvider; import org.apache.kafka.common.config.ConfigTransformer; import org.apache.kafka.common.config.ConfigTransformerResult; +import org.apache.kafka.connect.runtime.Herder.ConfigReloadAction; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; +import java.util.Locale; import java.util.Map; import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.ConcurrentMap; @@ -29,6 +34,8 @@ * retrieved TTL values. */ public class WorkerConfigTransformer { + private static final Logger log = LoggerFactory.getLogger(WorkerConfigTransformer.class); + private final Worker worker; private final ConfigTransformer configTransformer; private final ConcurrentMap> requests = new ConcurrentHashMap<>(); @@ -38,9 +45,25 @@ public WorkerConfigTransformer(Worker worker, Map config this.configTransformer = new ConfigTransformer(configProviders); } + public Map transform(Map configs) { + return transform(null, configs); + } + public Map transform(String connectorName, Map configs) { + if (configs == null) return null; ConfigTransformerResult result = configTransformer.transform(configs); - scheduleReload(connectorName, result.ttls()); + if (connectorName != null) { + String key = ConnectorConfig.CONFIG_RELOAD_ACTION_CONFIG; + String action = (String) ConfigDef.parseType(key, configs.get(key), ConfigDef.Type.STRING); + if (action == null) { + // The default action is "restart". + action = ConnectorConfig.CONFIG_RELOAD_ACTION_RESTART; + } + ConfigReloadAction reloadAction = ConfigReloadAction.valueOf(action.toUpperCase(Locale.ROOT)); + if (reloadAction == ConfigReloadAction.RESTART) { + scheduleReload(connectorName, result.ttls()); + } + } return result.data(); } @@ -51,21 +74,19 @@ private void scheduleReload(String connectorName, Map ttls) { } private void scheduleReload(String connectorName, String path, long ttl) { - Herder herder = worker.herder(); - if (herder.connectorConfigReloadAction(connectorName) == Herder.ConfigReloadAction.RESTART) { - Map connectorRequests = requests.get(connectorName); - if (connectorRequests == null) { - connectorRequests = new ConcurrentHashMap<>(); - requests.put(connectorName, connectorRequests); - } else { - HerderRequest previousRequest = connectorRequests.get(path); - if (previousRequest != null) { - // Delete previous request for ttl which is now stale - previousRequest.cancel(); - } + Map connectorRequests = requests.get(connectorName); + if (connectorRequests == null) { + connectorRequests = new ConcurrentHashMap<>(); + requests.put(connectorName, connectorRequests); + } else { + HerderRequest previousRequest = connectorRequests.get(path); + if (previousRequest != null) { + // Delete previous request for ttl which is now stale + previousRequest.cancel(); } - HerderRequest request = herder.restartConnector(ttl, connectorName, null); - connectorRequests.put(path, request); } + log.info("Scheduling a restart of connector {} in {} ms", connectorName, ttl); + HerderRequest request = worker.herder().restartConnector(ttl, connectorName, null); + connectorRequests.put(path, request); } } diff --git a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/WorkerConnector.java b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/WorkerConnector.java index 611e196d9de9d..55d4860b2e66b 100644 --- a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/WorkerConnector.java +++ b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/WorkerConnector.java @@ -89,7 +89,7 @@ public void requestTaskReconfiguration() { @Override public void raiseError(Exception e) { - log.error("{} Connector raised an error", this, e); + log.error("{} Connector raised an error", WorkerConnector.this, e); onFailure(e); ctx.raiseError(e); } diff --git a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/WorkerSinkTask.java b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/WorkerSinkTask.java index 47f8529e2d149..6e1ef04191bc6 100644 --- a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/WorkerSinkTask.java +++ b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/WorkerSinkTask.java @@ -288,6 +288,7 @@ protected void initializeAndStart() { if (SinkConnectorConfig.hasTopicsConfig(taskConfig)) { String[] topics = taskConfig.get(SinkTask.TOPICS_CONFIG).split(","); + Arrays.setAll(topics, i -> topics[i].trim()); consumer.subscribe(Arrays.asList(topics), new HandleRebalance()); log.debug("{} Initializing and starting task for topics {}", this, topics); } else { @@ -648,7 +649,7 @@ public void onPartitionsAssigned(Collection partitions) { long pos = consumer.position(tp); lastCommittedOffsets.put(tp, new OffsetAndMetadata(pos)); currentOffsets.put(tp, new OffsetAndMetadata(pos)); - log.debug("{} Assigned topic partition {} with offset {}", this, tp, pos); + log.debug("{} Assigned topic partition {} with offset {}", WorkerSinkTask.this, tp, pos); } sinkTaskMetricsGroup.assignedOffsets(currentOffsets); diff --git a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/WorkerSourceTask.java b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/WorkerSourceTask.java index 70d0cf9d7aea3..640cf3a2c8c5c 100644 --- a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/WorkerSourceTask.java +++ b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/WorkerSourceTask.java @@ -21,7 +21,6 @@ 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.errors.RetriableException; import org.apache.kafka.common.header.internals.RecordHeaders; import org.apache.kafka.common.metrics.Sensor; import org.apache.kafka.common.metrics.stats.Avg; @@ -31,6 +30,7 @@ import org.apache.kafka.common.metrics.stats.Value; import org.apache.kafka.common.utils.Time; import org.apache.kafka.connect.errors.ConnectException; +import org.apache.kafka.connect.errors.RetriableException; import org.apache.kafka.connect.header.Header; import org.apache.kafka.connect.header.Headers; import org.apache.kafka.connect.runtime.ConnectMetrics.MetricGroup; @@ -39,9 +39,9 @@ import org.apache.kafka.connect.runtime.errors.Stage; import org.apache.kafka.connect.source.SourceRecord; import org.apache.kafka.connect.source.SourceTask; +import org.apache.kafka.connect.storage.CloseableOffsetStorageReader; import org.apache.kafka.connect.storage.Converter; import org.apache.kafka.connect.storage.HeaderConverter; -import org.apache.kafka.connect.storage.OffsetStorageReader; import org.apache.kafka.connect.storage.OffsetStorageWriter; import org.apache.kafka.connect.util.ConnectUtils; import org.apache.kafka.connect.util.ConnectorTaskId; @@ -56,6 +56,7 @@ import java.util.concurrent.Future; import java.util.concurrent.TimeUnit; import java.util.concurrent.TimeoutException; +import java.util.concurrent.atomic.AtomicReference; /** * WorkerTask that uses a SourceTask to ingest data into Kafka. @@ -73,10 +74,11 @@ class WorkerSourceTask extends WorkerTask { private final HeaderConverter headerConverter; private final TransformationChain transformationChain; private KafkaProducer producer; - private final OffsetStorageReader offsetReader; + private final CloseableOffsetStorageReader offsetReader; private final OffsetStorageWriter offsetWriter; private final Time time; private final SourceTaskMetricsGroup sourceTaskMetricsGroup; + private final AtomicReference producerSendException; private List toSend; private boolean lastSendFailed; // Whether the last send failed *synchronously*, i.e. never made it into the producer's RecordAccumulator @@ -102,7 +104,7 @@ public WorkerSourceTask(ConnectorTaskId id, HeaderConverter headerConverter, TransformationChain transformationChain, KafkaProducer producer, - OffsetStorageReader offsetReader, + CloseableOffsetStorageReader offsetReader, OffsetStorageWriter offsetWriter, WorkerConfig workerConfig, ClusterConfigState configState, @@ -132,6 +134,7 @@ public WorkerSourceTask(ConnectorTaskId id, this.flushing = false; this.stopRequestedLatch = new CountDownLatch(1); this.sourceTaskMetricsGroup = new SourceTaskMetricsGroup(id, connectMetrics); + this.producerSendException = new AtomicReference<>(); } @Override @@ -168,6 +171,12 @@ protected void releaseResources() { sourceTaskMetricsGroup.close(); } + @Override + public void cancel() { + super.cancel(); + offsetReader.close(); + } + @Override public void stop() { super.stop(); @@ -214,6 +223,8 @@ public void execute() { continue; } + maybeThrowProducerSendException(); + if (toSend == null) { log.trace("{} Nothing to send to Kafka. Polling source for additional records", this); long start = time.milliseconds(); @@ -239,10 +250,19 @@ public void execute() { } } + private void maybeThrowProducerSendException() { + if (producerSendException.get() != null) { + throw new ConnectException( + "Unrecoverable exception from producer send callback", + producerSendException.get() + ); + } + } + protected List poll() throws InterruptedException { try { return task.poll(); - } catch (RetriableException e) { + } catch (RetriableException | org.apache.kafka.common.errors.RetriableException e) { log.warn("{} failed to poll records from SourceTask. Will retry operation.", this, e); // Do nothing. Let the framework poll whenever it's ready. return null; @@ -285,8 +305,10 @@ private ProducerRecord convertTransformedRecord(SourceRecord rec private boolean sendRecords() { int processed = 0; recordBatch(toSend.size()); - final SourceRecordWriteCounter counter = new SourceRecordWriteCounter(toSend.size(), sourceTaskMetricsGroup); + final SourceRecordWriteCounter counter = + toSend.size() > 0 ? new SourceRecordWriteCounter(toSend.size(), sourceTaskMetricsGroup) : null; for (final SourceRecord preTransformRecord : toSend) { + maybeThrowProducerSendException(); retryWithToleranceOperator.sourceRecord(preTransformRecord); final SourceRecord record = transformationChain.apply(preTransformRecord); @@ -321,26 +343,22 @@ private boolean sendRecords() { @Override public void onCompletion(RecordMetadata recordMetadata, Exception e) { if (e != null) { - // Given the default settings for zero data loss, this should basically never happen -- - // between "infinite" retries, indefinite blocking on full buffers, and "infinite" request - // timeouts, callbacks with exceptions should never be invoked in practice. If the - // user overrode these settings, the best we can do is notify them of the failure via - // logging. - log.error("{} failed to send record to {}: {}", this, topic, e); - log.debug("{} Failed record: {}", this, preTransformRecord); + log.error("{} failed to send record to {}:", WorkerSourceTask.this, topic, e); + log.debug("{} Failed record: {}", WorkerSourceTask.this, preTransformRecord); + producerSendException.compareAndSet(null, e); } else { + recordSent(producerRecord); + counter.completeRecord(); log.trace("{} Wrote record successfully: topic {} partition {} offset {}", - this, + WorkerSourceTask.this, recordMetadata.topic(), recordMetadata.partition(), recordMetadata.offset()); commitTaskRecord(preTransformRecord); } - recordSent(producerRecord); - counter.completeRecord(); } }); lastSendFailed = false; - } catch (RetriableException e) { + } catch (org.apache.kafka.common.errors.RetriableException e) { log.warn("{} Failed to send {}, backing off before retrying:", this, producerRecord, e); toSend = toSend.subList(processed, toSend.size()); lastSendFailed = true; @@ -454,9 +472,9 @@ public boolean commitOffsets() { @Override public void onCompletion(Throwable error, Void result) { if (error != null) { - log.error("{} Failed to flush offsets to storage: ", this, error); + log.error("{} Failed to flush offsets to storage: ", WorkerSourceTask.this, error); } else { - log.trace("{} Finished flushing offsets to storage", this); + log.trace("{} Finished flushing offsets to storage", WorkerSourceTask.this); } } }); diff --git a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/distributed/ClusterConfigState.java b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/distributed/ClusterConfigState.java index 9507706840fc2..fc6a50d2fc078 100644 --- a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/distributed/ClusterConfigState.java +++ b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/distributed/ClusterConfigState.java @@ -16,6 +16,7 @@ */ package org.apache.kafka.connect.runtime.distributed; +import org.apache.kafka.common.config.provider.ConfigProvider; import org.apache.kafka.connect.runtime.WorkerConfigTransformer; import org.apache.kafka.connect.runtime.TargetState; import org.apache.kafka.connect.util.ConnectorTaskId; @@ -110,7 +111,7 @@ public Set connectors() { * Get the configuration for a connector. The configuration will have been transformed by * {@link org.apache.kafka.common.config.ConfigTransformer} by having all variable * references replaced with the current values from external instances of - * {@link org.apache.kafka.common.config.ConfigProvider}, and may include secrets. + * {@link ConfigProvider}, and may include secrets. * @param connector name of the connector * @return a map containing configuration parameters */ @@ -122,6 +123,10 @@ public Map connectorConfig(String connector) { return configs; } + public Map rawConnectorConfig(String connector) { + return connectorConfigs.get(connector); + } + /** * Get the target state of the connector * @param connector name of the connector @@ -135,7 +140,7 @@ public TargetState targetState(String connector) { * Get the configuration for a task. The configuration will have been transformed by * {@link org.apache.kafka.common.config.ConfigTransformer} by having all variable * references replaced with the current values from external instances of - * {@link org.apache.kafka.common.config.ConfigProvider}, and may include secrets. + * {@link ConfigProvider}, and may include secrets. * @param task id of the task * @return a map containing configuration parameters */ @@ -147,16 +152,28 @@ public Map taskConfig(ConnectorTaskId task) { return configs; } + public Map rawTaskConfig(ConnectorTaskId task) { + return taskConfigs.get(task); + } + /** - * Get all task configs for a connector. + * Get all task configs for a connector. The configurations will have been transformed by + * {@link org.apache.kafka.common.config.ConfigTransformer} by having all variable + * references replaced with the current values from external instances of + * {@link ConfigProvider}, and may include secrets. * @param connector name of the connector * @return a list of task configurations */ public List> allTaskConfigs(String connector) { Map> taskConfigs = new TreeMap<>(); for (Map.Entry> taskConfigEntry : this.taskConfigs.entrySet()) { - if (taskConfigEntry.getKey().connector().equals(connector)) - taskConfigs.put(taskConfigEntry.getKey().task(), taskConfigEntry.getValue()); + if (taskConfigEntry.getKey().connector().equals(connector)) { + Map configs = taskConfigEntry.getValue(); + if (configTransformer != null) { + configs = configTransformer.transform(connector, configs); + } + taskConfigs.put(taskConfigEntry.getKey().task(), configs); + } } return new LinkedList<>(taskConfigs.values()); } diff --git a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/distributed/DistributedConfig.java b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/distributed/DistributedConfig.java index dc9017beeda75..af112a57c3b96 100644 --- a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/distributed/DistributedConfig.java +++ b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/distributed/DistributedConfig.java @@ -268,6 +268,11 @@ public class DistributedConfig extends WorkerConfig { STATUS_STORAGE_REPLICATION_FACTOR_CONFIG_DOC); } + @Override + public Integer getRebalanceTimeout() { + return getInt(DistributedConfig.REBALANCE_TIMEOUT_MS_CONFIG); + } + public DistributedConfig(Map props) { super(CONFIG, props); } diff --git a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/distributed/DistributedHerder.java b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/distributed/DistributedHerder.java index 5efb78a93e40d..b229102bc035d 100644 --- a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/distributed/DistributedHerder.java +++ b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/distributed/DistributedHerder.java @@ -61,7 +61,6 @@ import java.util.Collections; import java.util.HashSet; import java.util.List; -import java.util.Locale; import java.util.Map; import java.util.NavigableSet; import java.util.NoSuchElementException; @@ -452,7 +451,7 @@ public Void call() throws Exception { if (!configState.contains(connName)) { callback.onCompletion(new NotFoundException("Connector " + connName + " not found"), null); } else { - Map config = configState.connectorConfig(connName); + Map config = configState.rawConnectorConfig(connName); callback.onCompletion(null, new ConnectorInfo(connName, config, configState.tasks(connName), connectorTypeForClass(config.get(ConnectorConfig.CONNECTOR_CLASS_CONFIG)))); @@ -556,9 +555,9 @@ public Void call() throws Exception { // Note that we use the updated connector config despite the fact that we don't have an updated // snapshot yet. The existing task info should still be accurate. - Map map = configState.connectorConfig(connName); ConnectorInfo info = new ConnectorInfo(connName, config, configState.tasks(connName), - map == null ? null : connectorTypeForClass(map.get(ConnectorConfig.CONNECTOR_CLASS_CONFIG))); + // validateConnectorConfig have checked the existence of CONNECTOR_CLASS_CONFIG + connectorTypeForClass(config.get(ConnectorConfig.CONNECTOR_CLASS_CONFIG))); callback.onCompletion(null, new Created<>(!exists, info)); return null; } @@ -608,7 +607,7 @@ public Void call() throws Exception { List result = new ArrayList<>(); for (int i = 0; i < configState.taskCount(connName); i++) { ConnectorTaskId id = new ConnectorTaskId(connName, i); - result.add(new TaskInfo(id, configState.taskConfig(id))); + result.add(new TaskInfo(id, configState.rawTaskConfig(id))); } callback.onCompletion(null, result); } @@ -642,13 +641,6 @@ else if (!configState.contains(connName)) ); } - @Override - public ConfigReloadAction connectorConfigReloadAction(final String connName) { - return ConfigReloadAction.valueOf( - configState.connectorConfig(connName).get(ConnectorConfig.CONFIG_RELOAD_ACTION_CONFIG) - .toUpperCase(Locale.ROOT)); - } - @Override public void restartConnector(final String connName, final Callback callback) { restartConnector(0, connName, callback); @@ -1020,8 +1012,9 @@ private void reconfigureConnector(final String connName, final Callback cb } } if (changed) { + List> rawTaskProps = reverseTransform(connName, configState, taskProps); if (isLeader()) { - configBackingStore.putTaskConfigs(connName, taskProps); + configBackingStore.putTaskConfigs(connName, rawTaskProps); cb.onCompletion(null, null); } else { // We cannot forward the request on the same thread because this reconfiguration can happen as a result of connector @@ -1030,8 +1023,15 @@ private void reconfigureConnector(final String connName, final Callback cb @Override public void run() { try { - String reconfigUrl = RestServer.urlJoin(leaderUrl(), "/connectors/" + connName + "/tasks"); - RestClient.httpRequest(reconfigUrl, "POST", taskProps, null, config); + String leaderUrl = leaderUrl(); + if (leaderUrl == null || leaderUrl.trim().isEmpty()) { + cb.onCompletion(new ConnectException("Request to leader to " + + "reconfigure connector tasks failed " + + "because the URL of the leader's REST interface is empty!"), null); + return; + } + String reconfigUrl = RestServer.urlJoin(leaderUrl, "/connectors/" + connName + "/tasks"); + RestClient.httpRequest(reconfigUrl, "POST", null, rawTaskProps, null, config); cb.onCompletion(null, null); } catch (ConnectException e) { log.error("Request to leader to reconfigure connector tasks failed", e); diff --git a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/distributed/WorkerGroupMember.java b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/distributed/WorkerGroupMember.java index 525ce7e2de93d..eacc6f1d972b2 100644 --- a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/distributed/WorkerGroupMember.java +++ b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/distributed/WorkerGroupMember.java @@ -33,6 +33,7 @@ import org.apache.kafka.common.utils.AppInfoParser; import org.apache.kafka.common.utils.LogContext; import org.apache.kafka.common.utils.Time; +import org.apache.kafka.common.utils.Utils; import org.apache.kafka.connect.storage.ConfigBackingStore; import org.apache.kafka.connect.util.ConnectorTaskId; import org.slf4j.Logger; @@ -198,9 +199,9 @@ private void stop(boolean swallowException) { log.trace("Stopping the Connect group member."); AtomicReference firstException = new AtomicReference<>(); this.stopped = true; - ClientUtils.closeQuietly(coordinator, "coordinator", firstException); - ClientUtils.closeQuietly(metrics, "consumer metrics", firstException); - ClientUtils.closeQuietly(client, "consumer network client", firstException); + Utils.closeQuietly(coordinator, "coordinator", firstException); + Utils.closeQuietly(metrics, "consumer metrics", firstException); + Utils.closeQuietly(client, "consumer network client", firstException); AppInfoParser.unregisterAppInfo(JMX_PREFIX, clientId, metrics); if (firstException.get() != null && !swallowException) throw new KafkaException("Failed to stop the Connect group member", firstException.get()); diff --git a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/errors/DeadLetterQueueReporter.java b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/errors/DeadLetterQueueReporter.java index 9a8a9afb29bd8..231226997833d 100644 --- a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/errors/DeadLetterQueueReporter.java +++ b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/errors/DeadLetterQueueReporter.java @@ -22,14 +22,21 @@ import org.apache.kafka.clients.producer.KafkaProducer; import org.apache.kafka.clients.producer.ProducerRecord; import org.apache.kafka.common.errors.TopicExistsException; +import org.apache.kafka.common.header.Headers; import org.apache.kafka.common.record.RecordBatch; import org.apache.kafka.connect.errors.ConnectException; import org.apache.kafka.connect.runtime.SinkConnectorConfig; import org.apache.kafka.connect.runtime.WorkerConfig; +import org.apache.kafka.connect.util.ConnectorTaskId; import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import java.io.ByteArrayOutputStream; +import java.io.IOException; +import java.io.PrintStream; +import java.nio.charset.StandardCharsets; import java.util.Map; +import java.util.Objects; import java.util.concurrent.ExecutionException; import static java.util.Collections.singleton; @@ -44,22 +51,36 @@ public class DeadLetterQueueReporter implements ErrorReporter { private static final Logger log = LoggerFactory.getLogger(DeadLetterQueueReporter.class); - private static final short DLQ_MAX_DESIRED_REPLICATION_FACTOR = 3; private static final int DLQ_NUM_DESIRED_PARTITIONS = 1; + public static final String HEADER_PREFIX = "__connect.errors."; + public static final String ERROR_HEADER_ORIG_TOPIC = HEADER_PREFIX + "topic"; + public static final String ERROR_HEADER_ORIG_PARTITION = HEADER_PREFIX + "partition"; + public static final String ERROR_HEADER_ORIG_OFFSET = HEADER_PREFIX + "offset"; + public static final String ERROR_HEADER_CONNECTOR_NAME = HEADER_PREFIX + "connector.name"; + public static final String ERROR_HEADER_TASK_ID = HEADER_PREFIX + "task.id"; + public static final String ERROR_HEADER_STAGE = HEADER_PREFIX + "stage"; + public static final String ERROR_HEADER_EXECUTING_CLASS = HEADER_PREFIX + "class.name"; + public static final String ERROR_HEADER_EXCEPTION = HEADER_PREFIX + "exception.class.name"; + public static final String ERROR_HEADER_EXCEPTION_MESSAGE = HEADER_PREFIX + "exception.message"; + public static final String ERROR_HEADER_EXCEPTION_STACK_TRACE = HEADER_PREFIX + "exception.stacktrace"; + private final SinkConnectorConfig connConfig; + private final ConnectorTaskId connectorTaskId; + private final ErrorHandlingMetrics errorHandlingMetrics; private KafkaProducer kafkaProducer; - private ErrorHandlingMetrics errorHandlingMetrics; public static DeadLetterQueueReporter createAndSetup(WorkerConfig workerConfig, - SinkConnectorConfig connConfig, Map producerProps) { - String topic = connConfig.dlqTopicName(); + ConnectorTaskId id, + SinkConnectorConfig sinkConfig, Map producerProps, + ErrorHandlingMetrics errorHandlingMetrics) { + String topic = sinkConfig.dlqTopicName(); try (AdminClient admin = AdminClient.create(workerConfig.originals())) { if (!admin.listTopics().names().get().contains(topic)) { log.error("Topic {} doesn't exist. Will attempt to create topic.", topic); - NewTopic schemaTopicRequest = new NewTopic(topic, DLQ_NUM_DESIRED_PARTITIONS, DLQ_MAX_DESIRED_REPLICATION_FACTOR); + NewTopic schemaTopicRequest = new NewTopic(topic, DLQ_NUM_DESIRED_PARTITIONS, sinkConfig.dlqTopicReplicationFactor()); admin.createTopics(singleton(schemaTopicRequest)).all().get(); } } catch (InterruptedException e) { @@ -71,7 +92,7 @@ public static DeadLetterQueueReporter createAndSetup(WorkerConfig workerConfig, } KafkaProducer dlqProducer = new KafkaProducer<>(producerProps); - return new DeadLetterQueueReporter(dlqProducer, connConfig); + return new DeadLetterQueueReporter(dlqProducer, sinkConfig, id, errorHandlingMetrics); } /** @@ -80,13 +101,16 @@ public static DeadLetterQueueReporter createAndSetup(WorkerConfig workerConfig, * @param kafkaProducer a Kafka Producer to produce the original consumed records. */ // Visible for testing - DeadLetterQueueReporter(KafkaProducer kafkaProducer, SinkConnectorConfig connConfig) { + DeadLetterQueueReporter(KafkaProducer kafkaProducer, SinkConnectorConfig connConfig, + ConnectorTaskId id, ErrorHandlingMetrics errorHandlingMetrics) { + Objects.requireNonNull(kafkaProducer); + Objects.requireNonNull(connConfig); + Objects.requireNonNull(id); + Objects.requireNonNull(errorHandlingMetrics); + this.kafkaProducer = kafkaProducer; this.connConfig = connConfig; - } - - @Override - public void metrics(ErrorHandlingMetrics errorHandlingMetrics) { + this.connectorTaskId = id; this.errorHandlingMetrics = errorHandlingMetrics; } @@ -118,6 +142,10 @@ public void report(ProcessingContext context) { originalMessage.key(), originalMessage.value(), originalMessage.headers()); } + if (connConfig.isDlqContextHeadersEnabled()) { + populateContextHeaders(producerRecord, context); + } + this.kafkaProducer.send(producerRecord, (metadata, exception) -> { if (exception != null) { log.error("Could not produce message to dead letter queue. topic=" + dlqTopicName, exception); @@ -125,4 +153,56 @@ public void report(ProcessingContext context) { } }); } + + // Visible for testing + void populateContextHeaders(ProducerRecord producerRecord, ProcessingContext context) { + Headers headers = producerRecord.headers(); + if (context.consumerRecord() != null) { + headers.add(ERROR_HEADER_ORIG_TOPIC, toBytes(context.consumerRecord().topic())); + headers.add(ERROR_HEADER_ORIG_PARTITION, toBytes(context.consumerRecord().partition())); + headers.add(ERROR_HEADER_ORIG_OFFSET, toBytes(context.consumerRecord().offset())); + } + + headers.add(ERROR_HEADER_CONNECTOR_NAME, toBytes(connectorTaskId.connector())); + headers.add(ERROR_HEADER_TASK_ID, toBytes(String.valueOf(connectorTaskId.task()))); + headers.add(ERROR_HEADER_STAGE, toBytes(context.stage().name())); + headers.add(ERROR_HEADER_EXECUTING_CLASS, toBytes(context.executingClass().getName())); + if (context.error() != null) { + headers.add(ERROR_HEADER_EXCEPTION, toBytes(context.error().getClass().getName())); + headers.add(ERROR_HEADER_EXCEPTION_MESSAGE, toBytes(context.error().getMessage())); + byte[] trace; + if ((trace = stacktrace(context.error())) != null) { + headers.add(ERROR_HEADER_EXCEPTION_STACK_TRACE, trace); + } + } + } + + private byte[] stacktrace(Throwable error) { + ByteArrayOutputStream bos = new ByteArrayOutputStream(); + try { + PrintStream stream = new PrintStream(bos, true, "UTF-8"); + error.printStackTrace(stream); + bos.close(); + return bos.toByteArray(); + } catch (IOException e) { + log.error("Could not serialize stacktrace.", e); + } + return null; + } + + private byte[] toBytes(int value) { + return toBytes(String.valueOf(value)); + } + + private byte[] toBytes(long value) { + return toBytes(String.valueOf(value)); + } + + private byte[] toBytes(String value) { + if (value != null) { + return value.getBytes(StandardCharsets.UTF_8); + } else { + return null; + } + } } diff --git a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/errors/ErrorReporter.java b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/errors/ErrorReporter.java index f7df1b2d1a38f..58336163fbf4a 100644 --- a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/errors/ErrorReporter.java +++ b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/errors/ErrorReporter.java @@ -28,12 +28,4 @@ public interface ErrorReporter { */ void report(ProcessingContext context); - /** - * Provides the container for error handling metrics to implementations. This method will be called once the error - * reporter object is instantiated. - * - * @param errorHandlingMetrics metrics for error handling (cannot be null). - */ - void metrics(ErrorHandlingMetrics errorHandlingMetrics); - } diff --git a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/errors/LogReporter.java b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/errors/LogReporter.java index e81bd54756807..8b07adf8e4992 100644 --- a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/errors/LogReporter.java +++ b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/errors/LogReporter.java @@ -21,6 +21,8 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import java.util.Objects; + /** * Writes errors and their context to application logs. */ @@ -30,12 +32,16 @@ public class LogReporter implements ErrorReporter { private final ConnectorTaskId id; private final ConnectorConfig connConfig; + private final ErrorHandlingMetrics errorHandlingMetrics; - private ErrorHandlingMetrics errorHandlingMetrics; + public LogReporter(ConnectorTaskId id, ConnectorConfig connConfig, ErrorHandlingMetrics errorHandlingMetrics) { + Objects.requireNonNull(id); + Objects.requireNonNull(connConfig); + Objects.requireNonNull(errorHandlingMetrics); - public LogReporter(ConnectorTaskId id, ConnectorConfig connConfig) { this.id = id; this.connConfig = connConfig; + this.errorHandlingMetrics = errorHandlingMetrics; } /** @@ -57,11 +63,6 @@ public void report(ProcessingContext context) { errorHandlingMetrics.recordErrorLogged(); } - @Override - public void metrics(ErrorHandlingMetrics errorHandlingMetrics) { - this.errorHandlingMetrics = errorHandlingMetrics; - } - // Visible for testing String message(ProcessingContext context) { return String.format("Error encountered in task %s. %s", String.valueOf(id), diff --git a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/health/ConnectClusterStateImpl.java b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/health/ConnectClusterStateImpl.java index a0f7fdeea9fc8..43842127eac07 100644 --- a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/health/ConnectClusterStateImpl.java +++ b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/health/ConnectClusterStateImpl.java @@ -17,6 +17,7 @@ package org.apache.kafka.connect.runtime.health; +import org.apache.kafka.connect.errors.ConnectException; import org.apache.kafka.connect.health.ConnectClusterState; import org.apache.kafka.connect.health.ConnectorHealth; import org.apache.kafka.connect.health.ConnectorState; @@ -24,32 +25,35 @@ import org.apache.kafka.connect.health.TaskState; import org.apache.kafka.connect.runtime.Herder; import org.apache.kafka.connect.runtime.rest.entities.ConnectorStateInfo; -import org.apache.kafka.connect.util.Callback; +import org.apache.kafka.connect.util.FutureCallback; -import java.util.ArrayList; import java.util.Collection; import java.util.HashMap; import java.util.List; import java.util.Map; +import java.util.concurrent.ExecutionException; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.TimeoutException; public class ConnectClusterStateImpl implements ConnectClusterState { + + private final long herderRequestTimeoutMs; + private final Herder herder; - private Herder herder; - - public ConnectClusterStateImpl(Herder herder) { + public ConnectClusterStateImpl(long connectorsTimeoutMs, Herder herder) { + this.herderRequestTimeoutMs = connectorsTimeoutMs; this.herder = herder; } @Override public Collection connectors() { - final Collection connectors = new ArrayList<>(); - herder.connectors(new Callback>() { - @Override - public void onCompletion(Throwable error, Collection result) { - connectors.addAll(result); - } - }); - return connectors; + FutureCallback> connectorsCallback = new FutureCallback<>(); + herder.connectors(connectorsCallback); + try { + return connectorsCallback.get(herderRequestTimeoutMs, TimeUnit.MILLISECONDS); + } catch (InterruptedException | ExecutionException | TimeoutException e) { + throw new ConnectException("Failed to retrieve list of connectors", e); + } } @Override @@ -78,7 +82,7 @@ private Map taskStates(List st for (ConnectorStateInfo.TaskState state : states) { taskStates.put( state.id(), - new TaskState(state.id(), state.workerId(), state.state(), state.trace()) + new TaskState(state.id(), state.state(), state.workerId(), state.trace()) ); } return taskStates; diff --git a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/isolation/DelegatingClassLoader.java b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/isolation/DelegatingClassLoader.java index 1e5985175079c..eeca98658a38b 100644 --- a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/isolation/DelegatingClassLoader.java +++ b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/isolation/DelegatingClassLoader.java @@ -16,7 +16,7 @@ */ package org.apache.kafka.connect.runtime.isolation; -import org.apache.kafka.common.config.ConfigProvider; +import org.apache.kafka.common.config.provider.ConfigProvider; import org.apache.kafka.connect.components.Versioned; import org.apache.kafka.connect.connector.Connector; import org.apache.kafka.connect.rest.ConnectRestExtension; @@ -46,6 +46,7 @@ import java.util.ArrayList; import java.util.Arrays; import java.util.Collection; +import java.util.Enumeration; import java.util.HashMap; import java.util.Iterator; import java.util.List; @@ -56,10 +57,12 @@ import java.util.SortedSet; import java.util.TreeMap; import java.util.TreeSet; +import java.util.stream.Collectors; public class DelegatingClassLoader extends URLClassLoader { private static final Logger log = LoggerFactory.getLogger(DelegatingClassLoader.class); private static final String CLASSPATH_NAME = "classpath"; + private static final String UNDEFINED_VERSION = "undefined"; private final Map, ClassLoader>> pluginLoaders; private final Map aliases; @@ -72,6 +75,12 @@ public class DelegatingClassLoader extends URLClassLoader { private final List pluginPaths; private final Map activePaths; + private static final String MANIFEST_PREFIX = "META-INF/services/"; + private static final Class[] SERVICE_LOADER_PLUGINS = new Class[] {ConnectRestExtension.class, ConfigProvider.class}; + private static final Set PLUGIN_MANIFEST_FILES = + Arrays.stream(SERVICE_LOADER_PLUGINS).map(serviceLoaderPlugin -> MANIFEST_PREFIX + serviceLoaderPlugin.getName()) + .collect(Collectors.toSet()); + public DelegatingClassLoader(List pluginPaths, ClassLoader parent) { super(new URL[0], parent); this.pluginPaths = pluginPaths; @@ -87,7 +96,11 @@ public DelegatingClassLoader(List pluginPaths, ClassLoader parent) { } public DelegatingClassLoader(List pluginPaths) { - this(pluginPaths, ClassLoader.getSystemClassLoader()); + // Use as parent the classloader that loaded this class. In most cases this will be the + // System classloader. But this choice here provides additional flexibility in managed + // environments that control classloading differently (OSGi, Spring and others) and don't + // depend on the System classloader to load Connect's classes. + this(pluginPaths, DelegatingClassLoader.class.getClassLoader()); } public Set> connectors() { @@ -114,6 +127,25 @@ public Set> restExtensions() { return restExtensions; } + /** + * Retrieve the PluginClassLoader associated with a plugin class + * @param name The fully qualified class name of the plugin + * @return the PluginClassLoader that should be used to load this, or null if the plugin is not isolated. + */ + public PluginClassLoader pluginClassLoader(String name) { + if (!PluginUtils.shouldLoadInIsolation(name)) { + return null; + } + SortedMap, ClassLoader> inner = pluginLoaders.get(name); + if (inner == null) { + return null; + } + ClassLoader pluginLoader = inner.get(inner.lastKey()); + return pluginLoader instanceof PluginClassLoader + ? (PluginClassLoader) pluginLoader + : null; + } + public ClassLoader connectorLoader(Connector connector) { return connectorLoader(connector.getClass().getName()); } @@ -123,8 +155,8 @@ public ClassLoader connectorLoader(String connectorClassOrAlias) { String fullName = aliases.containsKey(connectorClassOrAlias) ? aliases.get(connectorClassOrAlias) : connectorClassOrAlias; - SortedMap, ClassLoader> inner = pluginLoaders.get(fullName); - if (inner == null) { + PluginClassLoader classLoader = pluginClassLoader(fullName); + if (classLoader == null) { log.error( "Plugin class loader for connector: '{}' was not found. Returning: {}", connectorClassOrAlias, @@ -132,7 +164,7 @@ public ClassLoader connectorLoader(String connectorClassOrAlias) { ); return this; } - return inner.get(inner.lastKey()); + return classLoader; } private static PluginClassLoader newPluginClassLoader( @@ -301,7 +333,7 @@ private PluginScanResult scanPluginPath( getPluginDesc(reflections, Converter.class, loader), getPluginDesc(reflections, HeaderConverter.class, loader), getPluginDesc(reflections, Transformation.class, loader), - getPluginDesc(reflections, ConfigProvider.class, loader), + getServiceLoaderPluginDesc(ConfigProvider.class, loader), getServiceLoaderPluginDesc(ConnectRestExtension.class, loader) ); } @@ -316,7 +348,7 @@ private Collection> getPluginDesc( Collection> result = new ArrayList<>(); for (Class plugin : plugins) { if (PluginUtils.isConcrete(plugin)) { - result.add(new PluginDesc<>(plugin, versionFor(plugin.newInstance()), loader)); + result.add(new PluginDesc<>(plugin, versionFor(plugin), loader)); } else { log.debug("Skipping {} as it is not concrete implementation", plugin); } @@ -324,36 +356,37 @@ private Collection> getPluginDesc( return result; } - private Collection> getServiceLoaderPluginDesc(Class klass, - ClassLoader loader) { - - ServiceLoader serviceLoader = ServiceLoader.load(klass, loader); + private Collection> getServiceLoaderPluginDesc(Class klass, ClassLoader loader) { + ClassLoader savedLoader = Plugins.compareAndSwapLoaders(loader); Collection> result = new ArrayList<>(); - for (T impl : serviceLoader) { - result.add(new PluginDesc<>(klass, versionFor(impl), loader)); + try { + ServiceLoader serviceLoader = ServiceLoader.load(klass, loader); + for (T pluginImpl : serviceLoader) { + result.add(new PluginDesc<>((Class) pluginImpl.getClass(), + versionFor(pluginImpl), loader)); + } + } finally { + Plugins.compareAndSwapLoaders(savedLoader); } return result; } private static String versionFor(T pluginImpl) { - return pluginImpl instanceof Versioned ? ((Versioned) pluginImpl).version() : "undefined"; + return pluginImpl instanceof Versioned ? ((Versioned) pluginImpl).version() : UNDEFINED_VERSION; + } + + private static String versionFor(Class pluginKlass) throws IllegalAccessException, InstantiationException { + // Temporary workaround until all the plugins are versioned. + return Connector.class.isAssignableFrom(pluginKlass) ? versionFor(pluginKlass.newInstance()) : UNDEFINED_VERSION; } @Override protected Class loadClass(String name, boolean resolve) throws ClassNotFoundException { - if (!PluginUtils.shouldLoadInIsolation(name)) { - // There are no paths in this classloader, will attempt to load with the parent. - return super.loadClass(name, resolve); - } - String fullName = aliases.containsKey(name) ? aliases.get(name) : name; - SortedMap, ClassLoader> inner = pluginLoaders.get(fullName); - if (inner != null) { - ClassLoader pluginLoader = inner.get(inner.lastKey()); + PluginClassLoader pluginLoader = pluginClassLoader(fullName); + if (pluginLoader != null) { log.trace("Retrieving loaded class '{}' from '{}'", fullName, pluginLoader); - return pluginLoader instanceof PluginClassLoader - ? ((PluginClassLoader) pluginLoader).loadClass(fullName, resolve) - : super.loadClass(fullName, resolve); + return pluginLoader.loadClass(fullName, resolve); } return super.loadClass(fullName, resolve); @@ -408,4 +441,31 @@ protected void scan(URL url) { } } } + + @Override + public URL getResource(String name) { + if (serviceLoaderManifestForPlugin(name)) { + // Default implementation of getResource searches the parent class loader and if not available/found, its own URL paths. + // This will enable thePluginClassLoader to limit its resource search only to its own URL paths. + return null; + } else { + return super.getResource(name); + } + } + + @Override + public Enumeration getResources(String name) throws IOException { + if (serviceLoaderManifestForPlugin(name)) { + // Default implementation of getResources searches the parent class loader and and also its own URL paths. This will enable the + // PluginClassLoader to limit its resource search to only its own URL paths. + return null; + } else { + return super.getResources(name); + } + } + + //Visible for testing + static boolean serviceLoaderManifestForPlugin(String name) { + return PLUGIN_MANIFEST_FILES.contains(name); + } } diff --git a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/isolation/PluginScanResult.java b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/isolation/PluginScanResult.java index 87b0b70c50359..ef077b3e7af19 100644 --- a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/isolation/PluginScanResult.java +++ b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/isolation/PluginScanResult.java @@ -16,7 +16,7 @@ */ package org.apache.kafka.connect.runtime.isolation; -import org.apache.kafka.common.config.ConfigProvider; +import org.apache.kafka.common.config.provider.ConfigProvider; import org.apache.kafka.connect.connector.Connector; import org.apache.kafka.connect.rest.ConnectRestExtension; import org.apache.kafka.connect.storage.Converter; diff --git a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/isolation/PluginType.java b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/isolation/PluginType.java index 906b85f70039f..2833b4c4ba0bf 100644 --- a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/isolation/PluginType.java +++ b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/isolation/PluginType.java @@ -16,7 +16,7 @@ */ package org.apache.kafka.connect.runtime.isolation; -import org.apache.kafka.common.config.ConfigProvider; +import org.apache.kafka.common.config.provider.ConfigProvider; import org.apache.kafka.connect.connector.Connector; import org.apache.kafka.connect.rest.ConnectRestExtension; import org.apache.kafka.connect.sink.SinkConnector; diff --git a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/isolation/PluginUtils.java b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/isolation/PluginUtils.java index d490bde4ed295..a9a273e42149f 100644 --- a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/isolation/PluginUtils.java +++ b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/isolation/PluginUtils.java @@ -122,12 +122,16 @@ public class PluginUtils { + "|org\\.slf4j" + ")\\..*$"; - private static final String WHITELIST = "^org\\.apache\\.kafka\\.connect\\.(?:" + private static final String WHITELIST = "^org\\.apache\\.kafka\\.(?:connect\\.(?:" + "transforms\\.(?!Transformation$).*" + "|json\\..*" + "|file\\..*" + "|converters\\..*" + "|storage\\.StringConverter" + + "|storage\\.SimpleHeaderConverter" + + "|rest\\.basic\\.auth\\.extension\\.BasicAuthSecurityRestExtension" + + ")" + + "|common\\.config\\.provider\\.(?!ConfigProvider$).*" + ")$"; private static final DirectoryStream.Filter PLUGIN_PATH_FILTER = new DirectoryStream diff --git a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/isolation/Plugins.java b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/isolation/Plugins.java index c89accd380573..5dd231e093937 100644 --- a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/isolation/Plugins.java +++ b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/isolation/Plugins.java @@ -17,9 +17,8 @@ package org.apache.kafka.connect.runtime.isolation; import org.apache.kafka.common.Configurable; -import org.apache.kafka.common.KafkaException; import org.apache.kafka.common.config.AbstractConfig; -import org.apache.kafka.common.config.ConfigProvider; +import org.apache.kafka.common.config.provider.ConfigProvider; import org.apache.kafka.common.utils.Utils; import org.apache.kafka.connect.components.Versioned; import org.apache.kafka.connect.connector.ConnectRecord; @@ -77,13 +76,37 @@ private static String pluginNames(Collection> plugins) { } protected static T newPlugin(Class klass) { + // KAFKA-8340: The thread classloader is used during static initialization and must be + // set to the plugin's classloader during instantiation + ClassLoader savedLoader = compareAndSwapLoaders(klass.getClassLoader()); try { return Utils.newInstance(klass); } catch (Throwable t) { throw new ConnectException("Instantiation error", t); + } finally { + compareAndSwapLoaders(savedLoader); } } + @SuppressWarnings("unchecked") + protected Class pluginClassFromConfig( + AbstractConfig config, + String propertyName, + Class pluginClass, + Collection> plugins + ) { + Class klass = config.getClass(propertyName); + if (pluginClass.isAssignableFrom(klass)) { + return (Class) klass; + } + throw new ConnectException( + "Failed to find any class that implements " + pluginClass.getSimpleName() + + " for the config " + + propertyName + ", available classes are: " + + pluginNames(plugins) + ); + } + @SuppressWarnings("unchecked") protected static Class pluginClass( DelegatingClassLoader loader, @@ -214,18 +237,17 @@ public Converter newConverter(AbstractConfig config, String classPropertyName, C // it does not represent an internal converter (which has a default available) return null; } - Converter plugin = null; + Class klass = null; switch (classLoaderUsage) { case CURRENT_CLASSLOADER: // Attempt to load first with the current classloader, and plugins as a fallback. // Note: we can't use config.getConfiguredInstance because Converter doesn't implement Configurable, and even if it did // we have to remove the property prefixes before calling config(...) and we still always want to call Converter.config. - plugin = getInstance(config, classPropertyName, Converter.class); + klass = pluginClassFromConfig(config, classPropertyName, Converter.class, delegatingLoader.converters()); break; case PLUGINS: // Attempt to load with the plugin class loader, which uses the current classloader as a fallback String converterClassOrAlias = config.getClass(classPropertyName).getName(); - Class klass; try { klass = pluginClass(delegatingLoader, converterClassOrAlias, Converter.class); } catch (ClassNotFoundException e) { @@ -235,11 +257,10 @@ public Converter newConverter(AbstractConfig config, String classPropertyName, C + pluginNames(delegatingLoader.converters()) ); } - plugin = newPlugin(klass); break; } - if (plugin == null) { - throw new ConnectException("Unable to instantiate the Converter specified in '" + classPropertyName + "'"); + if (klass == null) { + throw new ConnectException("Unable to initialize the Converter specified in '" + classPropertyName + "'"); } // Determine whether this is a key or value converter based upon the supplied property name ... @@ -249,13 +270,13 @@ public Converter newConverter(AbstractConfig config, String classPropertyName, C // Configure the Converter using only the old configuration mechanism ... String configPrefix = classPropertyName + "."; Map converterConfig = config.originalsWithPrefix(configPrefix); - log.debug("Configuring the {} converter with configuration:{}{}", - isKeyConverter ? "key" : "value", System.lineSeparator(), converterConfig); + log.debug("Configuring the {} converter with configuration keys:{}{}", + isKeyConverter ? "key" : "value", System.lineSeparator(), converterConfig.keySet()); // Have to override schemas.enable from true to false for internal JSON converters // Don't have to warn the user about anything since all deprecation warnings take place in the // WorkerConfig class - if (plugin instanceof JsonConverter && isInternalConverter(classPropertyName)) { + if (JsonConverter.class.isAssignableFrom(klass) && isInternalConverter(classPropertyName)) { // If they haven't explicitly specified values for internal.key.converter.schemas.enable // or internal.value.converter.schemas.enable, we can safely default them to false if (!converterConfig.containsKey(JsonConverterConfig.SCHEMAS_ENABLE_CONFIG)) { @@ -263,7 +284,14 @@ public Converter newConverter(AbstractConfig config, String classPropertyName, C } } - plugin.configure(converterConfig, isKeyConverter); + Converter plugin; + ClassLoader savedLoader = compareAndSwapLoaders(klass.getClassLoader()); + try { + plugin = newPlugin(klass); + plugin.configure(converterConfig, isKeyConverter); + } finally { + compareAndSwapLoaders(savedLoader); + } return plugin; } @@ -278,7 +306,7 @@ public Converter newConverter(AbstractConfig config, String classPropertyName, C * @throws ConnectException if the {@link HeaderConverter} implementation class could not be found */ public HeaderConverter newHeaderConverter(AbstractConfig config, String classPropertyName, ClassLoaderUsage classLoaderUsage) { - HeaderConverter plugin = null; + Class klass = null; switch (classLoaderUsage) { case CURRENT_CLASSLOADER: if (!config.originals().containsKey(classPropertyName)) { @@ -288,13 +316,12 @@ public HeaderConverter newHeaderConverter(AbstractConfig config, String classPro // Attempt to load first with the current classloader, and plugins as a fallback. // Note: we can't use config.getConfiguredInstance because we have to remove the property prefixes // before calling config(...) - plugin = getInstance(config, classPropertyName, HeaderConverter.class); + klass = pluginClassFromConfig(config, classPropertyName, HeaderConverter.class, delegatingLoader.headerConverters()); break; case PLUGINS: // Attempt to load with the plugin class loader, which uses the current classloader as a fallback. // Note that there will always be at least a default header converter for the worker String converterClassOrAlias = config.getClass(classPropertyName).getName(); - Class klass; try { klass = pluginClass( delegatingLoader, @@ -309,17 +336,24 @@ public HeaderConverter newHeaderConverter(AbstractConfig config, String classPro + pluginNames(delegatingLoader.headerConverters()) ); } - plugin = newPlugin(klass); } - if (plugin == null) { - throw new ConnectException("Unable to instantiate the Converter specified in '" + classPropertyName + "'"); + if (klass == null) { + throw new ConnectException("Unable to initialize the HeaderConverter specified in '" + classPropertyName + "'"); } String configPrefix = classPropertyName + "."; Map converterConfig = config.originalsWithPrefix(configPrefix); converterConfig.put(ConverterConfig.TYPE_CONFIG, ConverterType.HEADER.getName()); - log.debug("Configuring the header converter with configuration:{}{}", System.lineSeparator(), converterConfig); - plugin.configure(converterConfig); + log.debug("Configuring the header converter with configuration keys:{}{}", System.lineSeparator(), converterConfig.keySet()); + + HeaderConverter plugin; + ClassLoader savedLoader = compareAndSwapLoaders(klass.getClassLoader()); + try { + plugin = newPlugin(klass); + plugin.configure(converterConfig); + } finally { + compareAndSwapLoaders(savedLoader); + } return plugin; } @@ -330,16 +364,15 @@ public ConfigProvider newConfigProvider(AbstractConfig config, String providerPr // This configuration does not define the config provider via the specified property name return null; } - ConfigProvider plugin = null; + Class klass = null; switch (classLoaderUsage) { case CURRENT_CLASSLOADER: // Attempt to load first with the current classloader, and plugins as a fallback. - plugin = getInstance(config, classPropertyName, ConfigProvider.class); + klass = pluginClassFromConfig(config, classPropertyName, ConfigProvider.class, delegatingLoader.configProviders()); break; case PLUGINS: // Attempt to load with the plugin class loader, which uses the current classloader as a fallback String configProviderClassOrAlias = originalConfig.get(classPropertyName); - Class klass; try { klass = pluginClass(delegatingLoader, configProviderClassOrAlias, ConfigProvider.class); } catch (ClassNotFoundException e) { @@ -349,17 +382,24 @@ public ConfigProvider newConfigProvider(AbstractConfig config, String providerPr + pluginNames(delegatingLoader.configProviders()) ); } - plugin = newPlugin(klass); break; } - if (plugin == null) { - throw new ConnectException("Unable to instantiate the ConfigProvider specified in '" + classPropertyName + "'"); + if (klass == null) { + throw new ConnectException("Unable to initialize the ConfigProvider specified in '" + classPropertyName + "'"); } // Configure the ConfigProvider String configPrefix = providerPrefix + ".param."; Map configProviderConfig = config.originalsWithPrefix(configPrefix); - plugin.configure(configProviderConfig); + + ConfigProvider plugin; + ClassLoader savedLoader = compareAndSwapLoaders(klass.getClassLoader()); + try { + plugin = newPlugin(klass); + plugin.configure(configProviderConfig); + } finally { + compareAndSwapLoaders(savedLoader); + } return plugin; } @@ -393,42 +433,25 @@ public T newPlugin(String klassName, AbstractConfig config, Class pluginK + "name matches %s", pluginKlass, klassName); throw new ConnectException(msg); } - plugin = newPlugin(klass); - if (plugin == null) { - throw new ConnectException("Unable to instantiate '" + klassName + "'"); - } - if (plugin instanceof Versioned) { - Versioned versionedPlugin = (Versioned) plugin; - if (versionedPlugin.version() == null || versionedPlugin.version().trim().isEmpty()) { - throw new ConnectException("Version not defined for '" + klassName + "'"); + ClassLoader savedLoader = compareAndSwapLoaders(klass.getClassLoader()); + try { + plugin = newPlugin(klass); + if (plugin instanceof Versioned) { + Versioned versionedPlugin = (Versioned) plugin; + if (versionedPlugin.version() == null || versionedPlugin.version().trim() + .isEmpty()) { + throw new ConnectException("Version not defined for '" + klassName + "'"); + } } - } - if (plugin instanceof Configurable) { - ((Configurable) plugin).configure(config.originals()); + if (plugin instanceof Configurable) { + ((Configurable) plugin).configure(config.originals()); + } + } finally { + compareAndSwapLoaders(savedLoader); } return plugin; } - /** - * Get an instance of the give class specified by the given configuration key. - * - * @param key The configuration key for the class - * @param t The interface the class should implement - * @return A instance of the class - */ - private T getInstance(AbstractConfig config, String key, Class t) { - Class c = config.getClass(key); - if (c == null) { - return null; - } - // Instantiate the class, but we don't know if the class extends the supplied type - Object o = Utils.newInstance(c); - if (!t.isInstance(o)) { - throw new KafkaException(c.getName() + " is not an instance of " + t.getName()); - } - return t.cast(o); - } - public > Transformation newTranformations( String transformationClassOrAlias ) { diff --git a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/rest/RestClient.java b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/rest/RestClient.java index 15e8418a30c90..de11f262b3ea5 100644 --- a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/rest/RestClient.java +++ b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/rest/RestClient.java @@ -19,6 +19,7 @@ import com.fasterxml.jackson.core.type.TypeReference; import com.fasterxml.jackson.databind.ObjectMapper; +import javax.ws.rs.core.HttpHeaders; import org.apache.kafka.connect.runtime.WorkerConfig; import org.apache.kafka.connect.runtime.rest.entities.ErrorMessage; import org.apache.kafka.connect.runtime.rest.errors.ConnectRestException; @@ -50,12 +51,13 @@ public class RestClient { * * @param url HTTP connection will be established with this url. * @param method HTTP method ("GET", "POST", "PUT", etc.) + * @param headers HTTP headers from REST endpoint * @param requestBodyData Object to serialize as JSON and send in the request body. * @param responseFormat Expected format of the response to the HTTP request. * @param The type of the deserialized response to the HTTP request. * @return The deserialized response to the HTTP request, or null if no data is expected. */ - public static HttpResponse httpRequest(String url, String method, Object requestBodyData, + public static HttpResponse httpRequest(String url, String method, HttpHeaders headers, Object requestBodyData, TypeReference responseFormat, WorkerConfig config) { HttpClient client; @@ -82,6 +84,8 @@ public static HttpResponse httpRequest(String url, String method, Object req.method(method); req.accept("application/json"); req.agent("kafka-connect"); + addHeadersToRequest(headers, req); + if (serializedBody != null) { req.content(new StringContentProvider(serializedBody, StandardCharsets.UTF_8), "application/json"); } @@ -116,6 +120,21 @@ public static HttpResponse httpRequest(String url, String method, Object } } + + /** + * Extract headers from REST call and add to client request + * @param headers Headers from REST endpoint + * @param req The client request to modify + */ + private static void addHeadersToRequest(HttpHeaders headers, Request req) { + if (headers != null) { + String credentialAuthorization = headers.getHeaderString(HttpHeaders.AUTHORIZATION); + if (credentialAuthorization != null) { + req.header(HttpHeaders.AUTHORIZATION, credentialAuthorization); + } + } + } + /** * Convert response parameters from Jetty format (HttpFields) * @param httpFields diff --git a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/rest/RestServer.java b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/rest/RestServer.java index 5a589db8858ee..9ac6bb522cabc 100644 --- a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/rest/RestServer.java +++ b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/rest/RestServer.java @@ -17,7 +17,6 @@ package org.apache.kafka.connect.runtime.rest; import com.fasterxml.jackson.jaxrs.json.JacksonJsonProvider; - import org.apache.kafka.common.config.ConfigException; import org.apache.kafka.connect.errors.ConnectException; import org.apache.kafka.connect.rest.ConnectRestExtension; @@ -35,8 +34,8 @@ import org.eclipse.jetty.server.Server; import org.eclipse.jetty.server.ServerConnector; import org.eclipse.jetty.server.Slf4jRequestLog; +import org.eclipse.jetty.server.handler.ContextHandlerCollection; import org.eclipse.jetty.server.handler.DefaultHandler; -import org.eclipse.jetty.server.handler.HandlerCollection; import org.eclipse.jetty.server.handler.RequestLogHandler; import org.eclipse.jetty.server.handler.StatisticsHandler; import org.eclipse.jetty.servlet.FilterHolder; @@ -45,10 +44,13 @@ import org.eclipse.jetty.servlets.CrossOriginFilter; import org.eclipse.jetty.util.ssl.SslContextFactory; import org.glassfish.jersey.server.ResourceConfig; +import org.glassfish.jersey.server.ServerProperties; import org.glassfish.jersey.servlet.ServletContainer; import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import javax.servlet.DispatcherType; +import javax.ws.rs.core.UriBuilder; import java.io.IOException; import java.net.URI; import java.util.ArrayList; @@ -59,9 +61,6 @@ import java.util.regex.Matcher; import java.util.regex.Pattern; -import javax.servlet.DispatcherType; -import javax.ws.rs.core.UriBuilder; - /** * Embedded server for the REST API that provides the control plane for Kafka Connect workers. */ @@ -75,6 +74,7 @@ public class RestServer { private static final String PROTOCOL_HTTPS = "https"; private final WorkerConfig config; + private ContextHandlerCollection handlers; private Server jettyServer; private List connectRestExtensions = Collections.EMPTY_LIST; @@ -88,6 +88,7 @@ public RestServer(WorkerConfig config) { List listeners = parseListeners(); jettyServer = new Server(); + handlers = new ContextHandlerCollection(); createConnectors(listeners); } @@ -159,8 +160,28 @@ public Connector createConnector(String listener) { return connector; } - public void start(Herder herder) { - log.info("Starting REST server"); + public void initializeServer() { + log.info("Initializing REST server"); + + /* Needed for graceful shutdown as per `setStopTimeout` documentation */ + StatisticsHandler statsHandler = new StatisticsHandler(); + statsHandler.setHandler(handlers); + jettyServer.setHandler(statsHandler); + jettyServer.setStopTimeout(GRACEFUL_SHUTDOWN_TIMEOUT_MS); + jettyServer.setStopAtShutdown(true); + + try { + jettyServer.start(); + } catch (Exception e) { + throw new ConnectException("Unable to initialize REST server", e); + } + + log.info("REST server listening at " + jettyServer.getURI() + ", advertising URL " + advertisedUrl()); + } + + @SuppressWarnings("deprecation") + public void initializeResources(Herder herder) { + log.info("Initializing REST resources"); ResourceConfig resourceConfig = new ResourceConfig(); resourceConfig.register(new JacksonJsonProvider()); @@ -170,6 +191,7 @@ public void start(Herder herder) { resourceConfig.register(new ConnectorPluginsResource(herder)); resourceConfig.register(ConnectExceptionMapper.class); + resourceConfig.property(ServerProperties.WADL_FEATURE_DISABLE, true); registerRestExtensions(herder, resourceConfig); @@ -198,26 +220,19 @@ public void start(Herder herder) { requestLog.setLogLatency(true); requestLogHandler.setRequestLog(requestLog); - HandlerCollection handlers = new HandlerCollection(); handlers.setHandlers(new Handler[]{context, new DefaultHandler(), requestLogHandler}); - - /* Needed for graceful shutdown as per `setStopTimeout` documentation */ - StatisticsHandler statsHandler = new StatisticsHandler(); - statsHandler.setHandler(handlers); - jettyServer.setHandler(statsHandler); - jettyServer.setStopTimeout(GRACEFUL_SHUTDOWN_TIMEOUT_MS); - jettyServer.setStopAtShutdown(true); - try { - jettyServer.start(); + context.start(); } catch (Exception e) { - throw new ConnectException("Unable to start REST server", e); + throw new ConnectException("Unable to initialize REST resources", e); } - log.info("REST server listening at " + jettyServer.getURI() + ", advertising URL " + advertisedUrl()); + log.info("REST resources initialized; server is started and ready to handle requests"); } - + public URI serverUrl() { + return jettyServer.getURI(); + } public void stop() { log.info("Stopping REST server"); @@ -233,9 +248,8 @@ public void stop() { jettyServer.stop(); jettyServer.join(); } catch (Exception e) { - throw new ConnectException("Unable to stop REST server", e); - } finally { jettyServer.destroy(); + throw new ConnectException("Unable to stop REST server", e); } log.info("REST server stopped"); @@ -243,7 +257,8 @@ public void stop() { /** * Get the URL to advertise to other workers and clients. This uses the default connector from the embedded Jetty - * server, unless overrides for advertised hostname and/or port are provided via configs. + * server, unless overrides for advertised hostname and/or port are provided via configs. {@link #initializeServer()} + * must be invoked successfully before calling this method. */ public URI advertisedUrl() { UriBuilder builder = UriBuilder.fromUri(jettyServer.getURI()); @@ -261,7 +276,7 @@ else if (serverConnector != null && serverConnector.getHost() != null && serverC Integer advertisedPort = config.getInt(WorkerConfig.REST_ADVERTISED_PORT_CONFIG); if (advertisedPort != null) builder.port(advertisedPort); - else if (serverConnector != null) + else if (serverConnector != null && serverConnector.getPort() > 0) builder.port(serverConnector.getPort()); log.info("Advertised URI: {}", builder.build()); @@ -304,10 +319,18 @@ void registerRestExtensions(Herder herder, ResourceConfig resourceConfig) { config.getList(WorkerConfig.REST_EXTENSION_CLASSES_CONFIG), config, ConnectRestExtension.class); + long herderRequestTimeoutMs = ConnectorsResource.REQUEST_TIMEOUT_MS; + + Integer rebalanceTimeoutMs = config.getRebalanceTimeout(); + + if (rebalanceTimeoutMs != null) { + herderRequestTimeoutMs = Math.min(herderRequestTimeoutMs, rebalanceTimeoutMs.longValue()); + } + ConnectRestExtensionContext connectRestExtensionContext = new ConnectRestExtensionContextImpl( new ConnectRestConfigurable(resourceConfig), - new ConnectClusterStateImpl(herder) + new ConnectClusterStateImpl(herderRequestTimeoutMs, herder) ); for (ConnectRestExtension connectRestExtension : connectRestExtensions) { connectRestExtension.register(connectRestExtensionContext); @@ -322,4 +345,4 @@ public static String urlJoin(String base, String path) { return base + path; } -} \ No newline at end of file +} diff --git a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/rest/entities/ConnectorInfo.java b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/rest/entities/ConnectorInfo.java index 9a10d7488770f..f36ee74e980be 100644 --- a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/rest/entities/ConnectorInfo.java +++ b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/rest/entities/ConnectorInfo.java @@ -71,12 +71,13 @@ public boolean equals(Object o) { ConnectorInfo that = (ConnectorInfo) o; return Objects.equals(name, that.name) && Objects.equals(config, that.config) && - Objects.equals(tasks, that.tasks); + Objects.equals(tasks, that.tasks) && + Objects.equals(type, that.type); } @Override public int hashCode() { - return Objects.hash(name, config, tasks); + return Objects.hash(name, config, tasks, type); } } diff --git a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/rest/entities/ConnectorStateInfo.java b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/rest/entities/ConnectorStateInfo.java index 80192ca064489..6280473af964d 100644 --- a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/rest/entities/ConnectorStateInfo.java +++ b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/rest/entities/ConnectorStateInfo.java @@ -90,7 +90,10 @@ public String trace() { } public static class ConnectorState extends AbstractState { - public ConnectorState(String state, String worker, String msg) { + @JsonCreator + public ConnectorState(@JsonProperty("state") String state, + @JsonProperty("worker_id") String worker, + @JsonProperty("msg") String msg) { super(state, worker, msg); } } @@ -98,7 +101,11 @@ public ConnectorState(String state, String worker, String msg) { public static class TaskState extends AbstractState implements Comparable { private final int id; - public TaskState(int id, String state, String worker, String msg) { + @JsonCreator + public TaskState(@JsonProperty("id") int id, + @JsonProperty("state") String state, + @JsonProperty("worker_id") String worker, + @JsonProperty("msg") String msg) { super(state, worker, msg); this.id = id; } diff --git a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/rest/resources/ConnectorsResource.java b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/rest/resources/ConnectorsResource.java index e9661046d31e9..26a09ea3c43d9 100644 --- a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/rest/resources/ConnectorsResource.java +++ b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/rest/resources/ConnectorsResource.java @@ -17,6 +17,8 @@ package org.apache.kafka.connect.runtime.rest.resources; import com.fasterxml.jackson.core.type.TypeReference; + +import javax.ws.rs.core.HttpHeaders; import org.apache.kafka.connect.runtime.ConnectorConfig; import org.apache.kafka.connect.runtime.Herder; import org.apache.kafka.connect.runtime.WorkerConfig; @@ -44,6 +46,7 @@ import javax.ws.rs.PathParam; import javax.ws.rs.Produces; import javax.ws.rs.QueryParam; +import javax.ws.rs.core.Context; import javax.ws.rs.core.MediaType; import javax.ws.rs.core.Response; import javax.ws.rs.core.UriBuilder; @@ -65,7 +68,7 @@ public class ConnectorsResource { // session timeout to complete, during which we cannot serve some requests. Ideally we could reduce this, but // we need to consider all possible scenarios this could fail. It might be ok to fail with a timeout in rare cases, // but currently a worker simply leaving the group can take this long as well. - private static final long REQUEST_TIMEOUT_MS = 90 * 1000; + public static final long REQUEST_TIMEOUT_MS = 90 * 1000; private final Herder herder; private final WorkerConfig config; @@ -79,16 +82,18 @@ public ConnectorsResource(Herder herder, WorkerConfig config) { @GET @Path("/") - public Collection listConnectors(final @QueryParam("forward") Boolean forward) throws Throwable { + public Collection listConnectors(final @QueryParam("forward") Boolean forward, + final @Context HttpHeaders headers) throws Throwable { FutureCallback> cb = new FutureCallback<>(); herder.connectors(cb); - return completeOrForwardRequest(cb, "/connectors", "GET", null, new TypeReference>() { + return completeOrForwardRequest(cb, "/connectors", "GET", headers, null, new TypeReference>() { }, forward); } @POST @Path("/") public Response createConnector(final @QueryParam("forward") Boolean forward, + final @Context HttpHeaders headers, final CreateConnectorRequest createRequest) throws Throwable { // Trim leading and trailing whitespaces from the connector name, replace null with empty string // if no name element present to keep validation within validator (NonEmptyStringWithoutControlChars @@ -100,7 +105,7 @@ public Response createConnector(final @QueryParam("forward") Boolean forward, FutureCallback> cb = new FutureCallback<>(); herder.putConnectorConfig(name, configs, false, cb); - Herder.Created info = completeOrForwardRequest(cb, "/connectors", "POST", createRequest, + Herder.Created info = completeOrForwardRequest(cb, "/connectors", "POST", headers, createRequest, new TypeReference() { }, new CreatedConnectorInfoTranslator(), forward); URI location = UriBuilder.fromUri("/connectors").path(name).build(); @@ -110,19 +115,21 @@ public Response createConnector(final @QueryParam("forward") Boolean forward, @GET @Path("/{connector}") public ConnectorInfo getConnector(final @PathParam("connector") String connector, + final @Context HttpHeaders headers, final @QueryParam("forward") Boolean forward) throws Throwable { FutureCallback cb = new FutureCallback<>(); herder.connectorInfo(connector, cb); - return completeOrForwardRequest(cb, "/connectors/" + connector, "GET", null, forward); + return completeOrForwardRequest(cb, "/connectors/" + connector, "GET", headers, null, forward); } @GET @Path("/{connector}/config") public Map getConnectorConfig(final @PathParam("connector") String connector, + final @Context HttpHeaders headers, final @QueryParam("forward") Boolean forward) throws Throwable { FutureCallback> cb = new FutureCallback<>(); herder.connectorConfig(connector, cb); - return completeOrForwardRequest(cb, "/connectors/" + connector + "/config", "GET", null, forward); + return completeOrForwardRequest(cb, "/connectors/" + connector + "/config", "GET", headers, null, forward); } @GET @@ -134,6 +141,7 @@ public ConnectorStateInfo getConnectorStatus(final @PathParam("connector") Strin @PUT @Path("/{connector}/config") public Response putConnectorConfig(final @PathParam("connector") String connector, + final @Context HttpHeaders headers, final @QueryParam("forward") Boolean forward, final Map connectorConfig) throws Throwable { FutureCallback> cb = new FutureCallback<>(); @@ -141,7 +149,7 @@ public Response putConnectorConfig(final @PathParam("connector") String connecto herder.putConnectorConfig(connector, connectorConfig, true, cb); Herder.Created createdInfo = completeOrForwardRequest(cb, "/connectors/" + connector + "/config", - "PUT", connectorConfig, new TypeReference() { }, new CreatedConnectorInfoTranslator(), forward); + "PUT", headers, connectorConfig, new TypeReference() { }, new CreatedConnectorInfoTranslator(), forward); Response.ResponseBuilder response; if (createdInfo.created()) { URI location = UriBuilder.fromUri("/connectors").path(connector).build(); @@ -155,15 +163,16 @@ public Response putConnectorConfig(final @PathParam("connector") String connecto @POST @Path("/{connector}/restart") public void restartConnector(final @PathParam("connector") String connector, + final @Context HttpHeaders headers, final @QueryParam("forward") Boolean forward) throws Throwable { FutureCallback cb = new FutureCallback<>(); herder.restartConnector(connector, cb); - completeOrForwardRequest(cb, "/connectors/" + connector + "/restart", "POST", null, forward); + completeOrForwardRequest(cb, "/connectors/" + connector + "/restart", "POST", headers, null, forward); } @PUT @Path("/{connector}/pause") - public Response pauseConnector(@PathParam("connector") String connector) { + public Response pauseConnector(@PathParam("connector") String connector, final @Context HttpHeaders headers) { herder.pauseConnector(connector); return Response.accepted().build(); } @@ -178,26 +187,29 @@ public Response resumeConnector(@PathParam("connector") String connector) { @GET @Path("/{connector}/tasks") public List getTaskConfigs(final @PathParam("connector") String connector, + final @Context HttpHeaders headers, final @QueryParam("forward") Boolean forward) throws Throwable { FutureCallback> cb = new FutureCallback<>(); herder.taskConfigs(connector, cb); - return completeOrForwardRequest(cb, "/connectors/" + connector + "/tasks", "GET", null, new TypeReference>() { + return completeOrForwardRequest(cb, "/connectors/" + connector + "/tasks", "GET", headers, null, new TypeReference>() { }, forward); } @POST @Path("/{connector}/tasks") public void putTaskConfigs(final @PathParam("connector") String connector, + final @Context HttpHeaders headers, final @QueryParam("forward") Boolean forward, final List> taskConfigs) throws Throwable { FutureCallback cb = new FutureCallback<>(); herder.putTaskConfigs(connector, taskConfigs, cb); - completeOrForwardRequest(cb, "/connectors/" + connector + "/tasks", "POST", taskConfigs, forward); + completeOrForwardRequest(cb, "/connectors/" + connector + "/tasks", "POST", headers, taskConfigs, forward); } @GET @Path("/{connector}/tasks/{task}/status") public ConnectorStateInfo.TaskState getTaskStatus(final @PathParam("connector") String connector, + final @Context HttpHeaders headers, final @PathParam("task") Integer task) throws Throwable { return herder.taskStatus(new ConnectorTaskId(connector, task)); } @@ -206,20 +218,22 @@ public ConnectorStateInfo.TaskState getTaskStatus(final @PathParam("connector") @Path("/{connector}/tasks/{task}/restart") public void restartTask(final @PathParam("connector") String connector, final @PathParam("task") Integer task, + final @Context HttpHeaders headers, final @QueryParam("forward") Boolean forward) throws Throwable { FutureCallback cb = new FutureCallback<>(); ConnectorTaskId taskId = new ConnectorTaskId(connector, task); herder.restartTask(taskId, cb); - completeOrForwardRequest(cb, "/connectors/" + connector + "/tasks/" + task + "/restart", "POST", null, forward); + completeOrForwardRequest(cb, "/connectors/" + connector + "/tasks/" + task + "/restart", "POST", headers, null, forward); } @DELETE @Path("/{connector}") public void destroyConnector(final @PathParam("connector") String connector, + final @Context HttpHeaders headers, final @QueryParam("forward") Boolean forward) throws Throwable { FutureCallback> cb = new FutureCallback<>(); herder.deleteConnectorConfig(connector, cb); - completeOrForwardRequest(cb, "/connectors/" + connector, "DELETE", null, forward); + completeOrForwardRequest(cb, "/connectors/" + connector, "DELETE", headers, null, forward); } // Check whether the connector name from the url matches the one (if there is one) provided in the connectorconfig @@ -239,6 +253,7 @@ private void checkAndPutConnectorConfigName(String connectorName, Map T completeOrForwardRequest(FutureCallback cb, String path, String method, + HttpHeaders headers, Object body, TypeReference resultType, Translator translator, @@ -261,7 +276,7 @@ private T completeOrForwardRequest(FutureCallback cb, .build() .toString(); log.debug("Forwarding request {} {} {}", forwardUrl, method, body); - return translator.translate(RestClient.httpRequest(forwardUrl, method, body, resultType, config)); + return translator.translate(RestClient.httpRequest(forwardUrl, method, headers, body, resultType, config)); } else { // we should find the right target for the query within two hops, so if // we don't, it probably means that a rebalance has taken place. @@ -283,14 +298,14 @@ private T completeOrForwardRequest(FutureCallback cb, } } - private T completeOrForwardRequest(FutureCallback cb, String path, String method, Object body, + private T completeOrForwardRequest(FutureCallback cb, String path, String method, HttpHeaders headers, Object body, TypeReference resultType, Boolean forward) throws Throwable { - return completeOrForwardRequest(cb, path, method, body, resultType, new IdentityTranslator(), forward); + return completeOrForwardRequest(cb, path, method, headers, body, resultType, new IdentityTranslator(), forward); } - private T completeOrForwardRequest(FutureCallback cb, String path, String method, + private T completeOrForwardRequest(FutureCallback cb, String path, String method, HttpHeaders headers, Object body, Boolean forward) throws Throwable { - return completeOrForwardRequest(cb, path, method, body, null, new IdentityTranslator(), forward); + return completeOrForwardRequest(cb, path, method, headers, body, null, new IdentityTranslator(), forward); } private interface Translator { diff --git a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/standalone/StandaloneHerder.java b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/standalone/StandaloneHerder.java index 20c6a24d3841a..95b53e5d73070 100644 --- a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/standalone/StandaloneHerder.java +++ b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/standalone/StandaloneHerder.java @@ -42,7 +42,6 @@ import java.util.ArrayList; import java.util.Collection; import java.util.List; -import java.util.Locale; import java.util.Map; import java.util.Objects; import java.util.concurrent.Executors; @@ -135,7 +134,7 @@ public synchronized void connectorInfo(String connName, Callback private ConnectorInfo createConnectorInfo(String connector) { if (!configState.contains(connector)) return null; - Map config = configState.connectorConfig(connector); + Map config = configState.rawConnectorConfig(connector); return new ConnectorInfo(connector, config, configState.tasks(connector), connectorTypeForClass(config.get(ConnectorConfig.CONNECTOR_CLASS_CONFIG))); } @@ -201,7 +200,9 @@ public synchronized void putConnectorConfig(String connName, created = true; } - if (!startConnector(config)) { + configBackingStore.putConnectorConfig(connName, config); + + if (!startConnector(connName)) { callback.onCompletion(new ConnectException("Failed to start connector: " + connName), null); return; } @@ -231,7 +232,7 @@ public synchronized void taskConfigs(String connName, Callback> c List result = new ArrayList<>(); for (ConnectorTaskId taskId : configState.tasks(connName)) - result.add(new TaskInfo(taskId, configState.taskConfig(taskId))); + result.add(new TaskInfo(taskId, configState.rawTaskConfig(taskId))); callback.onCompletion(null, result); } @@ -258,21 +259,13 @@ public synchronized void restartTask(ConnectorTaskId taskId, Callback cb) cb.onCompletion(new ConnectException("Failed to start task: " + taskId), null); } - @Override - public ConfigReloadAction connectorConfigReloadAction(final String connName) { - return ConfigReloadAction.valueOf( - configState.connectorConfig(connName).get(ConnectorConfig.CONFIG_RELOAD_ACTION_CONFIG) - .toUpperCase(Locale.ROOT)); - } - @Override public synchronized void restartConnector(String connName, Callback cb) { if (!configState.contains(connName)) cb.onCompletion(new NotFoundException("Connector " + connName + " not found", null), null); - Map config = configState.connectorConfig(connName); worker.stopConnector(connName); - if (startConnector(config)) + if (startConnector(connName)) cb.onCompletion(null, null); else cb.onCompletion(new ConnectException("Failed to start connector: " + connName), null); @@ -290,9 +283,7 @@ public void run() { return new StandaloneHerderRequest(requestSeqNum.incrementAndGet(), future); } - private boolean startConnector(Map connectorProps) { - String connName = connectorProps.get(ConnectorConfig.NAME_CONFIG); - configBackingStore.putConnectorConfig(connName, connectorProps); + private boolean startConnector(String connName) { Map connConfigs = configState.connectorConfig(connName); TargetState targetState = configState.targetState(connName); return worker.startConnector(connName, connConfigs, new HerderConnectorContext(this, connName), this, targetState); @@ -336,7 +327,8 @@ private void updateConnectorTasks(String connName) { if (!newTaskConfigs.equals(oldTaskConfigs)) { removeConnectorTasks(connName); - configBackingStore.putTaskConfigs(connName, newTaskConfigs); + List> rawTaskConfigs = reverseTransform(connName, configState, newTaskConfigs); + configBackingStore.putTaskConfigs(connName, rawTaskConfigs); createConnectorTasks(connName, configState.targetState(connName)); } } diff --git a/connect/runtime/src/main/java/org/apache/kafka/connect/storage/CloseableOffsetStorageReader.java b/connect/runtime/src/main/java/org/apache/kafka/connect/storage/CloseableOffsetStorageReader.java new file mode 100644 index 0000000000000..b90273936bfb0 --- /dev/null +++ b/connect/runtime/src/main/java/org/apache/kafka/connect/storage/CloseableOffsetStorageReader.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.connect.storage; + +import java.io.Closeable; +import java.util.Collection; +import java.util.Map; +import java.util.concurrent.Future; + +public interface CloseableOffsetStorageReader extends Closeable, OffsetStorageReader { + + /** + * {@link Future#cancel(boolean) Cancel} all outstanding offset read requests, and throw an + * exception in all current and future calls to {@link #offsets(Collection)} and + * {@link #offset(Map)}. This is useful for unblocking task threads which need to shut down but + * are blocked on offset reads. + */ + void close(); +} diff --git a/connect/runtime/src/main/java/org/apache/kafka/connect/storage/KafkaConfigBackingStore.java b/connect/runtime/src/main/java/org/apache/kafka/connect/storage/KafkaConfigBackingStore.java index ea196650c5e1b..740cfc99a74a0 100644 --- a/connect/runtime/src/main/java/org/apache/kafka/connect/storage/KafkaConfigBackingStore.java +++ b/connect/runtime/src/main/java/org/apache/kafka/connect/storage/KafkaConfigBackingStore.java @@ -296,7 +296,7 @@ public boolean contains(String connector) { */ @Override public void putConnectorConfig(String connector, Map properties) { - log.debug("Writing connector configuration {} for connector {} configuration", properties, connector); + log.debug("Writing connector configuration for connector '{}'", connector); Struct connectConfig = new Struct(CONNECTOR_CONFIGURATION_V0); connectConfig.put("properties", properties); byte[] serializedConfig = converter.fromConnectData(topic, CONNECTOR_CONFIGURATION_V0, connectConfig); @@ -309,7 +309,7 @@ public void putConnectorConfig(String connector, Map properties) */ @Override public void removeConnectorConfig(String connector) { - log.debug("Removing connector configuration for connector {}", connector); + log.debug("Removing connector configuration for connector '{}'", connector); try { configLog.send(CONNECTOR_KEY(connector), null); configLog.send(TARGET_STATE_KEY(connector), null); @@ -363,7 +363,7 @@ public void putTaskConfigs(String connector, List> configs) Struct connectConfig = new Struct(TASK_CONFIGURATION_V0); connectConfig.put("properties", taskConfig); byte[] serializedConfig = converter.fromConnectData(topic, TASK_CONFIGURATION_V0, connectConfig); - log.debug("Writing configuration for task " + index + " configuration: " + taskConfig); + log.debug("Writing configuration for connector '{}' task {}", connector, index); ConnectorTaskId connectorTaskId = new ConnectorTaskId(connector, index); configLog.send(TASK_KEY(connectorTaskId), serializedConfig); index++; @@ -380,7 +380,7 @@ public void putTaskConfigs(String connector, List> configs) Struct connectConfig = new Struct(CONNECTOR_TASKS_COMMIT_V0); connectConfig.put("tasks", taskCount); byte[] serializedConfig = converter.fromConnectData(topic, CONNECTOR_TASKS_COMMIT_V0, connectConfig); - log.debug("Writing commit for connector " + connector + " with " + taskCount + " tasks."); + log.debug("Writing commit for connector '{}' with {} tasks.", connector, taskCount); configLog.send(COMMIT_TASKS_KEY(connector), serializedConfig); // Read to end to ensure all the commit messages have been written @@ -487,17 +487,17 @@ public void onCompletion(Throwable error, ConsumerRecord record) } Object targetState = ((Map) value.value()).get("state"); if (!(targetState instanceof String)) { - log.error("Invalid data for target state for connector ({}): 'state' field should be a Map but is {}", + log.error("Invalid data for target state for connector '{}': 'state' field should be a Map but is {}", connectorName, targetState == null ? null : targetState.getClass()); return; } try { TargetState state = TargetState.valueOf((String) targetState); - log.debug("Setting target state for connector {} to {}", connectorName, targetState); + log.debug("Setting target state for connector '{}' to {}", connectorName, targetState); connectorTargetStates.put(connectorName, state); } catch (IllegalArgumentException e) { - log.error("Invalid target state for connector ({}): {}", connectorName, targetState); + log.error("Invalid target state for connector '{}': {}", connectorName, targetState); return; } } @@ -514,22 +514,22 @@ public void onCompletion(Throwable error, ConsumerRecord record) synchronized (lock) { if (value.value() == null) { // Connector deletion will be written as a null value - log.info("Removed connector " + connectorName + " due to null configuration. This is usually intentional and does not indicate an issue."); + log.info("Successfully processed removal of connector '{}'", connectorName); connectorConfigs.remove(connectorName); removed = true; } else { // Connector configs can be applied and callbacks invoked immediately if (!(value.value() instanceof Map)) { - log.error("Found connector configuration (" + record.key() + ") in wrong format: " + value.value().getClass()); + log.error("Found configuration for connector '{}' in wrong format: {}", record.key(), value.value().getClass()); return; } Object newConnectorConfig = ((Map) value.value()).get("properties"); if (!(newConnectorConfig instanceof Map)) { - log.error("Invalid data for connector config ({}): properties field should be a Map but is {}", connectorName, - newConnectorConfig == null ? null : newConnectorConfig.getClass()); + log.error("Invalid data for config for connector '{}': 'properties' field should be a Map but is {}", + connectorName, newConnectorConfig == null ? null : newConnectorConfig.getClass()); return; } - log.debug("Updating configuration for connector " + connectorName + " configuration: " + newConnectorConfig); + log.debug("Updating configuration for connector '{}'", connectorName); connectorConfigs.put(connectorName, (Map) newConnectorConfig); // Set the initial state of the connector to STARTED, which ensures that any connectors @@ -548,17 +548,21 @@ public void onCompletion(Throwable error, ConsumerRecord record) synchronized (lock) { ConnectorTaskId taskId = parseTaskId(record.key()); if (taskId == null) { - log.error("Ignoring task configuration because " + record.key() + " couldn't be parsed as a task config key"); + log.error("Ignoring task configuration because {} couldn't be parsed as a task config key", record.key()); + return; + } + if (value.value() == null) { + log.error("Ignoring task configuration for task {} because it is unexpectedly null", taskId); return; } if (!(value.value() instanceof Map)) { - log.error("Ignoring task configuration for task " + taskId + " because it is in the wrong format: " + value.value()); + log.error("Ignoring task configuration for task {} because the value is not a Map but is {}", taskId, value.value().getClass()); return; } Object newTaskConfig = ((Map) value.value()).get("properties"); if (!(newTaskConfig instanceof Map)) { - log.error("Invalid data for task config (" + taskId + "): properties filed should be a Map but is " + newTaskConfig.getClass()); + log.error("Invalid data for config of task {} 'properties' field should be a Map but is {}", taskId, newTaskConfig.getClass()); return; } @@ -567,7 +571,7 @@ public void onCompletion(Throwable error, ConsumerRecord record) deferred = new HashMap<>(); deferredTaskUpdates.put(taskId.connector(), deferred); } - log.debug("Storing new config for task " + taskId + " this will wait for a commit message before the new config will take effect. New config: " + newTaskConfig); + log.debug("Storing new config for task {}; this will wait for a commit message before the new config will take effect.", taskId); deferred.put(taskId, (Map) newTaskConfig); } } else if (record.key().startsWith(COMMIT_TASKS_PREFIX)) { @@ -596,7 +600,7 @@ public void onCompletion(Throwable error, ConsumerRecord record) // resolve this (i.e., get the connector to recommit its configuration). This inconsistent state is // exposed in the snapshots provided via ClusterConfigState so they are easy to handle. if (!(value.value() instanceof Map)) { // Schema-less, so we get maps instead of structs - log.error("Ignoring connector tasks configuration commit for connector " + connectorName + " because it is in the wrong format: " + value.value()); + log.error("Ignoring connector tasks configuration commit for connector '{}' because it is in the wrong format: {}", connectorName, value.value()); return; } Map> deferred = deferredTaskUpdates.get(connectorName); @@ -611,7 +615,7 @@ public void onCompletion(Throwable error, ConsumerRecord record) // historical data, in which case we would not have applied any updates yet and there will be no // task config data already committed for the connector, so we shouldn't have to clear any data // out. All we need to do is add the flag marking it inconsistent. - log.debug("We have an incomplete set of task configs for connector " + connectorName + " probably due to compaction. So we are not doing anything with the new configuration."); + log.debug("We have an incomplete set of task configs for connector '{}' probably due to compaction. So we are not doing anything with the new configuration.", connectorName); inconsistent.add(connectorName); } else { if (deferred != null) { @@ -632,7 +636,7 @@ public void onCompletion(Throwable error, ConsumerRecord record) if (started) updateListener.onTaskConfigUpdate(updatedTasks); } else { - log.error("Discarding config update record with invalid key: " + record.key()); + log.error("Discarding config update record with invalid key: {}", record.key()); } } diff --git a/connect/runtime/src/main/java/org/apache/kafka/connect/storage/KafkaOffsetBackingStore.java b/connect/runtime/src/main/java/org/apache/kafka/connect/storage/KafkaOffsetBackingStore.java index fb8ad97b48ddb..ea002bda2d738 100644 --- a/connect/runtime/src/main/java/org/apache/kafka/connect/storage/KafkaOffsetBackingStore.java +++ b/connect/runtime/src/main/java/org/apache/kafka/connect/storage/KafkaOffsetBackingStore.java @@ -118,9 +118,8 @@ public void stop() { } @Override - public Future> get(final Collection keys, - final Callback> callback) { - ConvertingFutureCallback> future = new ConvertingFutureCallback>(callback) { + public Future> get(final Collection keys) { + ConvertingFutureCallback> future = new ConvertingFutureCallback>() { @Override public Map convert(Void result) { Map values = new HashMap<>(); @@ -230,6 +229,4 @@ public synchronized Void get(long timeout, TimeUnit unit) throws InterruptedExce return null; } } - - } diff --git a/connect/runtime/src/main/java/org/apache/kafka/connect/storage/MemoryOffsetBackingStore.java b/connect/runtime/src/main/java/org/apache/kafka/connect/storage/MemoryOffsetBackingStore.java index ab8130b6ef776..72439e7d687b3 100644 --- a/connect/runtime/src/main/java/org/apache/kafka/connect/storage/MemoryOffsetBackingStore.java +++ b/connect/runtime/src/main/java/org/apache/kafka/connect/storage/MemoryOffsetBackingStore.java @@ -75,9 +75,7 @@ public void stop() { } @Override - public Future> get( - final Collection keys, - final Callback> callback) { + public Future> get(final Collection keys) { return executor.submit(new Callable>() { @Override public Map call() throws Exception { @@ -85,8 +83,6 @@ public Map call() throws Exception { for (ByteBuffer key : keys) { result.put(key, data.get(key)); } - if (callback != null) - callback.onCompletion(null, result); return result; } }); diff --git a/connect/runtime/src/main/java/org/apache/kafka/connect/storage/OffsetBackingStore.java b/connect/runtime/src/main/java/org/apache/kafka/connect/storage/OffsetBackingStore.java index 9998164ddf5bf..1e4375b7d8eff 100644 --- a/connect/runtime/src/main/java/org/apache/kafka/connect/storage/OffsetBackingStore.java +++ b/connect/runtime/src/main/java/org/apache/kafka/connect/storage/OffsetBackingStore.java @@ -53,12 +53,9 @@ public interface OffsetBackingStore { /** * 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 */ - Future> get( - Collection keys, - Callback> callback); + Future> get(Collection keys); /** * Set the specified keys and values. @@ -66,8 +63,7 @@ Future> get( * @param callback callback to invoke on completion * @return void future for the operation */ - Future set(Map values, - Callback callback); + Future set(Map values, Callback callback); /** * Configure class with the given key-value pairs diff --git a/connect/runtime/src/main/java/org/apache/kafka/connect/storage/OffsetStorageReaderImpl.java b/connect/runtime/src/main/java/org/apache/kafka/connect/storage/OffsetStorageReaderImpl.java index 9f926dc5040aa..a1eea43103a39 100644 --- a/connect/runtime/src/main/java/org/apache/kafka/connect/storage/OffsetStorageReaderImpl.java +++ b/connect/runtime/src/main/java/org/apache/kafka/connect/storage/OffsetStorageReaderImpl.java @@ -26,20 +26,27 @@ import java.util.Collection; import java.util.Collections; import java.util.HashMap; +import java.util.HashSet; import java.util.Map; +import java.util.Set; +import java.util.concurrent.CancellationException; +import java.util.concurrent.Future; +import java.util.concurrent.atomic.AtomicBoolean; /** * 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 { +public class OffsetStorageReaderImpl implements CloseableOffsetStorageReader { 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; + private final AtomicBoolean closed; + private final Set>> offsetReadFutures; public OffsetStorageReaderImpl(OffsetBackingStore backingStore, String namespace, Converter keyConverter, Converter valueConverter) { @@ -47,6 +54,8 @@ public OffsetStorageReaderImpl(OffsetBackingStore backingStore, String namespace this.namespace = namespace; this.keyConverter = keyConverter; this.valueConverter = valueConverter; + this.closed = new AtomicBoolean(false); + this.offsetReadFutures = new HashSet<>(); } @Override @@ -76,7 +85,30 @@ public Map, Map> offsets(Collection serialized value from backing store Map raw; try { - raw = backingStore.get(serializedToOriginal.keySet(), null).get(); + Future> offsetReadFuture; + synchronized (offsetReadFutures) { + if (closed.get()) { + throw new ConnectException( + "Offset reader is closed. This is likely because the task has already been " + + "scheduled to stop but has taken longer than the graceful shutdown " + + "period to do so."); + } + offsetReadFuture = backingStore.get(serializedToOriginal.keySet()); + offsetReadFutures.add(offsetReadFuture); + } + + try { + raw = offsetReadFuture.get(); + } catch (CancellationException e) { + throw new ConnectException( + "Offset reader closed while attempting to read offsets. This is likely because " + + "the task was been scheduled to stop but has taken longer than the " + + "graceful shutdown period to do so."); + } finally { + synchronized (offsetReadFutures) { + offsetReadFutures.remove(offsetReadFuture); + } + } } catch (Exception e) { log.error("Failed to fetch offsets from namespace {}: ", namespace, e); throw new ConnectException("Failed to fetch offsets.", e); @@ -108,4 +140,19 @@ public Map, Map> offsets(Collection> offsetReadFuture : offsetReadFutures) { + try { + offsetReadFuture.cancel(true); + } catch (Throwable t) { + log.error("Failed to cancel offset read future", t); + } + } + offsetReadFutures.clear(); + } + } + } } diff --git a/connect/runtime/src/main/java/org/apache/kafka/connect/util/ConvertingFutureCallback.java b/connect/runtime/src/main/java/org/apache/kafka/connect/util/ConvertingFutureCallback.java index d5abed9385cc0..e15c38ea4c4ae 100644 --- a/connect/runtime/src/main/java/org/apache/kafka/connect/util/ConvertingFutureCallback.java +++ b/connect/runtime/src/main/java/org/apache/kafka/connect/util/ConvertingFutureCallback.java @@ -16,6 +16,9 @@ */ package org.apache.kafka.connect.util; +import org.apache.kafka.connect.errors.ConnectException; + +import java.util.concurrent.CancellationException; import java.util.concurrent.CountDownLatch; import java.util.concurrent.ExecutionException; import java.util.concurrent.Future; @@ -24,10 +27,15 @@ public abstract class ConvertingFutureCallback implements Callback, Future { - private Callback underlying; - private CountDownLatch finishedLatch; - private T result = null; - private Throwable exception = null; + private final Callback underlying; + private final CountDownLatch finishedLatch; + private volatile T result = null; + private volatile Throwable exception = null; + private volatile boolean cancelled = false; + + public ConvertingFutureCallback() { + this(null); + } public ConvertingFutureCallback(Callback underlying) { this.underlying = underlying; @@ -38,21 +46,46 @@ public ConvertingFutureCallback(Callback underlying) { @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(); + synchronized (this) { + if (isDone()) { + return; + } + + if (error != null) { + this.exception = error; + } else { + this.result = convert(result); + } + + if (underlying != null) + underlying.onCompletion(error, this.result); + finishedLatch.countDown(); + } } @Override - public boolean cancel(boolean b) { + public boolean cancel(boolean mayInterruptIfRunning) { + synchronized (this) { + if (isDone()) { + return false; + } + if (mayInterruptIfRunning) { + this.cancelled = true; + finishedLatch.countDown(); + return true; + } + } + try { + finishedLatch.await(); + } catch (InterruptedException e) { + throw new ConnectException("Interrupted while waiting for task to complete", e); + } return false; } @Override public boolean isCancelled() { - return false; + return cancelled; } @Override @@ -75,6 +108,9 @@ public T get(long l, TimeUnit timeUnit) } private T result() throws ExecutionException { + if (cancelled) { + throw new CancellationException(); + } if (exception != null) { throw new ExecutionException(exception); } diff --git a/connect/runtime/src/main/java/org/apache/kafka/connect/util/KafkaBasedLog.java b/connect/runtime/src/main/java/org/apache/kafka/connect/util/KafkaBasedLog.java index de1ceb3be1006..4a1af0adbe709 100644 --- a/connect/runtime/src/main/java/org/apache/kafka/connect/util/KafkaBasedLog.java +++ b/connect/runtime/src/main/java/org/apache/kafka/connect/util/KafkaBasedLog.java @@ -28,6 +28,7 @@ import org.apache.kafka.common.KafkaException; import org.apache.kafka.common.PartitionInfo; import org.apache.kafka.common.TopicPartition; +import org.apache.kafka.common.errors.TimeoutException; import org.apache.kafka.common.errors.WakeupException; import org.apache.kafka.common.utils.Time; import org.apache.kafka.common.utils.Utils; @@ -145,6 +146,10 @@ public void start() { partitions.add(new TopicPartition(partition.topic(), partition.partition())); consumer.assign(partitions); + // Always consume from the beginning of all partitions. Necessary to ensure that we don't use committed offsets + // when a 'group.id' is specified (if offsets happen to have been committed unexpectedly). + consumer.seekToBeginning(partitions); + readToLogEnd(); thread = new WorkThread(); @@ -307,6 +312,10 @@ public void run() { try { readToLogEnd(); log.trace("Finished read to end log for topic {}", topic); + } catch (TimeoutException e) { + log.warn("Timeout while reading log to end for topic '{}'. Retrying automatically. " + + "This may occur when brokers are unavailable or unreachable. Reason: {}", topic, e.getMessage()); + continue; } catch (WakeupException 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. diff --git a/connect/runtime/src/main/java/org/apache/kafka/connect/util/TopicAdmin.java b/connect/runtime/src/main/java/org/apache/kafka/connect/util/TopicAdmin.java index ad21561baf259..72a5981217231 100644 --- a/connect/runtime/src/main/java/org/apache/kafka/connect/util/TopicAdmin.java +++ b/connect/runtime/src/main/java/org/apache/kafka/connect/util/TopicAdmin.java @@ -22,6 +22,7 @@ import org.apache.kafka.clients.admin.NewTopic; import org.apache.kafka.common.KafkaFuture; import org.apache.kafka.common.errors.ClusterAuthorizationException; +import org.apache.kafka.common.errors.TopicAuthorizationException; import org.apache.kafka.common.errors.TimeoutException; import org.apache.kafka.common.errors.TopicExistsException; import org.apache.kafka.common.errors.UnsupportedVersionException; @@ -246,6 +247,12 @@ public Set createTopics(NewTopic... topics) { topicNameList, bootstrapServers); return Collections.emptySet(); } + if (cause instanceof TopicAuthorizationException) { + log.debug("Not authorized to create topic(s) '{}'." + + " Falling back to assume topic(s) exist or will be auto-created by the broker.", + topicNameList, bootstrapServers); + return Collections.emptySet(); + } if (cause instanceof TimeoutException) { // Timed out waiting for the operation to complete throw new ConnectException("Timed out while checking for or creating topic(s) '" + topicNameList + "'." + diff --git a/connect/api/src/test/java/org/apache/kafka/connect/storage/DoubleConverterTest.java b/connect/runtime/src/test/java/org/apache/kafka/connect/converters/DoubleConverterTest.java similarity index 97% rename from connect/api/src/test/java/org/apache/kafka/connect/storage/DoubleConverterTest.java rename to connect/runtime/src/test/java/org/apache/kafka/connect/converters/DoubleConverterTest.java index 1744083944525..acc3ddedc6815 100644 --- a/connect/api/src/test/java/org/apache/kafka/connect/storage/DoubleConverterTest.java +++ b/connect/runtime/src/test/java/org/apache/kafka/connect/converters/DoubleConverterTest.java @@ -14,7 +14,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.kafka.connect.storage; +package org.apache.kafka.connect.converters; import org.apache.kafka.common.serialization.DoubleSerializer; import org.apache.kafka.common.serialization.Serializer; diff --git a/connect/api/src/test/java/org/apache/kafka/connect/storage/FloatConverterTest.java b/connect/runtime/src/test/java/org/apache/kafka/connect/converters/FloatConverterTest.java similarity index 97% rename from connect/api/src/test/java/org/apache/kafka/connect/storage/FloatConverterTest.java rename to connect/runtime/src/test/java/org/apache/kafka/connect/converters/FloatConverterTest.java index 57a18602d1e40..e95ff56d9cfc8 100644 --- a/connect/api/src/test/java/org/apache/kafka/connect/storage/FloatConverterTest.java +++ b/connect/runtime/src/test/java/org/apache/kafka/connect/converters/FloatConverterTest.java @@ -14,7 +14,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.kafka.connect.storage; +package org.apache.kafka.connect.converters; import org.apache.kafka.common.serialization.FloatSerializer; import org.apache.kafka.common.serialization.Serializer; diff --git a/connect/api/src/test/java/org/apache/kafka/connect/storage/IntegerConverterTest.java b/connect/runtime/src/test/java/org/apache/kafka/connect/converters/IntegerConverterTest.java similarity index 97% rename from connect/api/src/test/java/org/apache/kafka/connect/storage/IntegerConverterTest.java rename to connect/runtime/src/test/java/org/apache/kafka/connect/converters/IntegerConverterTest.java index 33fbe600be31a..0c9ed28d9da46 100644 --- a/connect/api/src/test/java/org/apache/kafka/connect/storage/IntegerConverterTest.java +++ b/connect/runtime/src/test/java/org/apache/kafka/connect/converters/IntegerConverterTest.java @@ -14,7 +14,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.kafka.connect.storage; +package org.apache.kafka.connect.converters; import org.apache.kafka.common.serialization.IntegerSerializer; import org.apache.kafka.common.serialization.Serializer; diff --git a/connect/api/src/test/java/org/apache/kafka/connect/storage/LongConverterTest.java b/connect/runtime/src/test/java/org/apache/kafka/connect/converters/LongConverterTest.java similarity index 97% rename from connect/api/src/test/java/org/apache/kafka/connect/storage/LongConverterTest.java rename to connect/runtime/src/test/java/org/apache/kafka/connect/converters/LongConverterTest.java index 8f41bb5511db6..35d26b70e892c 100644 --- a/connect/api/src/test/java/org/apache/kafka/connect/storage/LongConverterTest.java +++ b/connect/runtime/src/test/java/org/apache/kafka/connect/converters/LongConverterTest.java @@ -14,7 +14,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.kafka.connect.storage; +package org.apache.kafka.connect.converters; import org.apache.kafka.common.serialization.LongSerializer; import org.apache.kafka.common.serialization.Serializer; diff --git a/connect/api/src/test/java/org/apache/kafka/connect/storage/NumberConverterTest.java b/connect/runtime/src/test/java/org/apache/kafka/connect/converters/NumberConverterTest.java similarity index 98% rename from connect/api/src/test/java/org/apache/kafka/connect/storage/NumberConverterTest.java rename to connect/runtime/src/test/java/org/apache/kafka/connect/converters/NumberConverterTest.java index 2936a7197898a..2bd07325b8a02 100644 --- a/connect/api/src/test/java/org/apache/kafka/connect/storage/NumberConverterTest.java +++ b/connect/runtime/src/test/java/org/apache/kafka/connect/converters/NumberConverterTest.java @@ -14,7 +14,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.kafka.connect.storage; +package org.apache.kafka.connect.converters; import org.apache.kafka.common.serialization.Serializer; import org.apache.kafka.connect.data.Schema; diff --git a/connect/api/src/test/java/org/apache/kafka/connect/storage/ShortConverterTest.java b/connect/runtime/src/test/java/org/apache/kafka/connect/converters/ShortConverterTest.java similarity index 97% rename from connect/api/src/test/java/org/apache/kafka/connect/storage/ShortConverterTest.java rename to connect/runtime/src/test/java/org/apache/kafka/connect/converters/ShortConverterTest.java index 871f39833dea0..d1237c9ca0442 100644 --- a/connect/api/src/test/java/org/apache/kafka/connect/storage/ShortConverterTest.java +++ b/connect/runtime/src/test/java/org/apache/kafka/connect/converters/ShortConverterTest.java @@ -14,7 +14,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.kafka.connect.storage; +package org.apache.kafka.connect.converters; import org.apache.kafka.common.serialization.Serializer; import org.apache.kafka.common.serialization.ShortSerializer; diff --git a/connect/runtime/src/test/java/org/apache/kafka/connect/integration/ConnectorHandle.java b/connect/runtime/src/test/java/org/apache/kafka/connect/integration/ConnectorHandle.java new file mode 100644 index 0000000000000..e59691b843d09 --- /dev/null +++ b/connect/runtime/src/test/java/org/apache/kafka/connect/integration/ConnectorHandle.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.connect.integration; + +import org.apache.kafka.connect.errors.DataException; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.util.Collection; +import java.util.Map; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.CountDownLatch; +import java.util.concurrent.TimeUnit; + +/** + * A handle to a connector executing in a Connect cluster. + */ +public class ConnectorHandle { + + private static final Logger log = LoggerFactory.getLogger(ConnectorHandle.class); + + private final String connectorName; + private final Map taskHandles = new ConcurrentHashMap<>(); + + private CountDownLatch recordsRemainingLatch; + private int expectedRecords = -1; + + public ConnectorHandle(String connectorName) { + this.connectorName = connectorName; + } + + /** + * Get or create a task handle for a given task id. The task need not be created when this method is called. If the + * handle is called before the task is created, the task will bind to the handle once it starts (or restarts). + * + * @param taskId the task id + * @return a non-null {@link TaskHandle} + */ + public TaskHandle taskHandle(String taskId) { + return taskHandles.computeIfAbsent(taskId, k -> new TaskHandle(this, taskId)); + } + + public Collection tasks() { + return taskHandles.values(); + } + + /** + * Delete the task handle for this task id. + * + * @param taskId the task id. + */ + public void deleteTask(String taskId) { + log.info("Removing handle for {} task in connector {}", taskId, connectorName); + taskHandles.remove(taskId); + } + + /** + * Set the number of expected records for this task. + * + * @param expectedRecords number of records + */ + public void expectedRecords(int expectedRecords) { + this.expectedRecords = expectedRecords; + this.recordsRemainingLatch = new CountDownLatch(expectedRecords); + } + + /** + * Record a message arrival at the connector. + */ + public void record() { + if (recordsRemainingLatch != null) { + recordsRemainingLatch.countDown(); + } + } + + /** + * Wait for this task to receive the expected number of records. + * + * @param consumeMaxDurationMs max duration to wait for records + * @throws InterruptedException if another threads interrupts this one while waiting for records + */ + public void awaitRecords(int consumeMaxDurationMs) throws InterruptedException { + if (recordsRemainingLatch == null || expectedRecords < 0) { + throw new IllegalStateException("expectedRecords() was not set for this task?"); + } + if (!recordsRemainingLatch.await(consumeMaxDurationMs, TimeUnit.MILLISECONDS)) { + String msg = String.format("Insufficient records seen by connector %s in %d millis. Records expected=%d, actual=%d", + connectorName, + consumeMaxDurationMs, + expectedRecords, + expectedRecords - recordsRemainingLatch.getCount()); + throw new DataException(msg); + } + } + + @Override + public String toString() { + return "ConnectorHandle{" + + "connectorName='" + connectorName + '\'' + + '}'; + } +} diff --git a/connect/runtime/src/test/java/org/apache/kafka/connect/integration/ErrorHandlingIntegrationTest.java b/connect/runtime/src/test/java/org/apache/kafka/connect/integration/ErrorHandlingIntegrationTest.java new file mode 100644 index 0000000000000..5f7cfc93082ea --- /dev/null +++ b/connect/runtime/src/test/java/org/apache/kafka/connect/integration/ErrorHandlingIntegrationTest.java @@ -0,0 +1,252 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.kafka.connect.integration; + +import org.apache.kafka.clients.consumer.ConsumerRecord; +import org.apache.kafka.clients.consumer.ConsumerRecords; +import org.apache.kafka.common.config.ConfigDef; +import org.apache.kafka.common.header.Headers; +import org.apache.kafka.connect.connector.ConnectRecord; +import org.apache.kafka.connect.errors.RetriableException; +import org.apache.kafka.connect.runtime.rest.entities.ConnectorStateInfo; +import org.apache.kafka.connect.storage.StringConverter; +import org.apache.kafka.connect.transforms.Transformation; +import org.apache.kafka.connect.util.clusters.EmbeddedConnectCluster; +import org.apache.kafka.test.IntegrationTest; +import org.junit.After; +import org.junit.Before; +import org.junit.Test; +import org.junit.experimental.categories.Category; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.IOException; +import java.util.HashMap; +import java.util.Map; + +import static org.apache.kafka.connect.runtime.ConnectorConfig.CONNECTOR_CLASS_CONFIG; +import static org.apache.kafka.connect.runtime.ConnectorConfig.ERRORS_LOG_ENABLE_CONFIG; +import static org.apache.kafka.connect.runtime.ConnectorConfig.ERRORS_LOG_INCLUDE_MESSAGES_CONFIG; +import static org.apache.kafka.connect.runtime.ConnectorConfig.ERRORS_RETRY_TIMEOUT_CONFIG; +import static org.apache.kafka.connect.runtime.ConnectorConfig.ERRORS_TOLERANCE_CONFIG; +import static org.apache.kafka.connect.runtime.ConnectorConfig.KEY_CONVERTER_CLASS_CONFIG; +import static org.apache.kafka.connect.runtime.ConnectorConfig.TASKS_MAX_CONFIG; +import static org.apache.kafka.connect.runtime.ConnectorConfig.TRANSFORMS_CONFIG; +import static org.apache.kafka.connect.runtime.ConnectorConfig.VALUE_CONVERTER_CLASS_CONFIG; +import static org.apache.kafka.connect.runtime.SinkConnectorConfig.DLQ_CONTEXT_HEADERS_ENABLE_CONFIG; +import static org.apache.kafka.connect.runtime.SinkConnectorConfig.DLQ_TOPIC_NAME_CONFIG; +import static org.apache.kafka.connect.runtime.SinkConnectorConfig.DLQ_TOPIC_REPLICATION_FACTOR_CONFIG; +import static org.apache.kafka.connect.runtime.SinkConnectorConfig.TOPICS_CONFIG; +import static org.apache.kafka.connect.runtime.errors.DeadLetterQueueReporter.ERROR_HEADER_EXCEPTION; +import static org.apache.kafka.connect.runtime.errors.DeadLetterQueueReporter.ERROR_HEADER_EXCEPTION_MESSAGE; +import static org.apache.kafka.connect.runtime.errors.DeadLetterQueueReporter.ERROR_HEADER_ORIG_TOPIC; +import static org.apache.kafka.test.TestUtils.waitForCondition; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertTrue; +import static org.junit.Assert.fail; + +/** + * Integration test for the different error handling policies in Connect (namely, retry policies, skipping bad records, + * and dead letter queues). + */ +@Category(IntegrationTest.class) +public class ErrorHandlingIntegrationTest { + + private static final Logger log = LoggerFactory.getLogger(ErrorHandlingIntegrationTest.class); + + private static final String DLQ_TOPIC = "my-connector-errors"; + private static final String CONNECTOR_NAME = "error-conn"; + private static final String TASK_ID = "error-conn-0"; + private static final int NUM_RECORDS_PRODUCED = 20; + private static final int EXPECTED_CORRECT_RECORDS = 19; + private static final int EXPECTED_INCORRECT_RECORDS = 1; + private static final int NUM_TASKS = 1; + private static final int CONNECTOR_SETUP_DURATION_MS = 5000; + private static final int CONSUME_MAX_DURATION_MS = 5000; + + private EmbeddedConnectCluster connect; + private ConnectorHandle connectorHandle; + + @Before + public void setup() throws IOException { + // setup Connect cluster with defaults + connect = new EmbeddedConnectCluster.Builder().build(); + + // start Connect cluster + connect.start(); + + // get connector handles before starting test. + connectorHandle = RuntimeHandles.get().connectorHandle(CONNECTOR_NAME); + } + + @After + public void close() { + RuntimeHandles.get().deleteConnector(CONNECTOR_NAME); + connect.stop(); + } + + @Test + public void testSkipRetryAndDLQWithHeaders() throws Exception { + // create test topic + connect.kafka().createTopic("test-topic"); + + // setup connector config + Map props = new HashMap<>(); + props.put(CONNECTOR_CLASS_CONFIG, "MonitorableSink"); + props.put(TASKS_MAX_CONFIG, String.valueOf(NUM_TASKS)); + props.put(TOPICS_CONFIG, "test-topic"); + props.put(KEY_CONVERTER_CLASS_CONFIG, StringConverter.class.getName()); + props.put(VALUE_CONVERTER_CLASS_CONFIG, StringConverter.class.getName()); + props.put(TRANSFORMS_CONFIG, "failing_transform"); + props.put("transforms.failing_transform.type", FaultyPassthrough.class.getName()); + + // log all errors, along with message metadata + props.put(ERRORS_LOG_ENABLE_CONFIG, "true"); + props.put(ERRORS_LOG_INCLUDE_MESSAGES_CONFIG, "true"); + + // produce bad messages into dead letter queue + props.put(DLQ_TOPIC_NAME_CONFIG, DLQ_TOPIC); + props.put(DLQ_CONTEXT_HEADERS_ENABLE_CONFIG, "true"); + props.put(DLQ_TOPIC_REPLICATION_FACTOR_CONFIG, "1"); + + // tolerate all erros + props.put(ERRORS_TOLERANCE_CONFIG, "all"); + + // retry for up to one second + props.put(ERRORS_RETRY_TIMEOUT_CONFIG, "1000"); + + // set expected records to successfully reach the task + connectorHandle.taskHandle(TASK_ID).expectedRecords(EXPECTED_CORRECT_RECORDS); + + connect.configureConnector(CONNECTOR_NAME, props); + + waitForCondition(this::checkForPartitionAssignment, + CONNECTOR_SETUP_DURATION_MS, + "Connector task was not assigned a partition."); + + // produce some strings into test topic + for (int i = 0; i < NUM_RECORDS_PRODUCED; i++) { + connect.kafka().produce("test-topic", "key-" + i, "value-" + i); + } + + // consume all records from test topic + log.info("Consuming records from test topic"); + int i = 0; + for (ConsumerRecord rec : connect.kafka().consume(NUM_RECORDS_PRODUCED, CONSUME_MAX_DURATION_MS, "test-topic")) { + String k = new String(rec.key()); + String v = new String(rec.value()); + log.debug("Consumed record (key='{}', value='{}') from topic {}", k, v, rec.topic()); + assertEquals("Unexpected key", k, "key-" + i); + assertEquals("Unexpected value", v, "value-" + i); + i++; + } + + // wait for records to reach the task + connectorHandle.taskHandle(TASK_ID).awaitRecords(CONSUME_MAX_DURATION_MS); + + // consume failed records from dead letter queue topic + log.info("Consuming records from test topic"); + ConsumerRecords messages = connect.kafka().consume(EXPECTED_INCORRECT_RECORDS, CONSUME_MAX_DURATION_MS, DLQ_TOPIC); + for (ConsumerRecord recs : messages) { + log.debug("Consumed record (key={}, value={}) from dead letter queue topic {}", + new String(recs.key()), new String(recs.value()), DLQ_TOPIC); + assertTrue(recs.headers().toArray().length > 0); + assertValue("test-topic", recs.headers(), ERROR_HEADER_ORIG_TOPIC); + assertValue(RetriableException.class.getName(), recs.headers(), ERROR_HEADER_EXCEPTION); + assertValue("Error when value='value-7'", recs.headers(), ERROR_HEADER_EXCEPTION_MESSAGE); + } + + connect.deleteConnector(CONNECTOR_NAME); + } + + /** + * Check if a partition was assigned to each task. This method swallows exceptions since it is invoked from a + * {@link org.apache.kafka.test.TestUtils#waitForCondition} that will throw an error if this method continued + * to return false after the specified duration has elapsed. + * + * @return true if each task was assigned a partition each, false if this was not true or an error occurred when + * executing this operation. + */ + private boolean checkForPartitionAssignment() { + try { + ConnectorStateInfo info = connect.connectorStatus(CONNECTOR_NAME); + return info != null && info.tasks().size() == NUM_TASKS + && connectorHandle.taskHandle(TASK_ID).partitionsAssigned() == 1; + } catch (Exception e) { + // Log the exception and return that the partitions were not assigned + log.error("Could not check connector state info.", e); + return false; + } + } + + private void assertValue(String expected, Headers headers, String headerKey) { + byte[] actual = headers.lastHeader(headerKey).value(); + if (expected == null && actual == null) { + return; + } + if (expected == null || actual == null) { + fail(); + } + assertEquals(expected, new String(actual)); + } + + public static class FaultyPassthrough> implements Transformation { + + static final ConfigDef CONFIG_DEF = new ConfigDef(); + + /** + * An arbitrary id which causes this transformation to fail with a {@link RetriableException}, but succeeds + * on subsequent attempt. + */ + static final int BAD_RECORD_VAL_RETRIABLE = 4; + + /** + * An arbitrary id which causes this transformation to fail with a {@link RetriableException}. + */ + static final int BAD_RECORD_VAL = 7; + + private boolean shouldFail = true; + + @Override + public R apply(R record) { + String badValRetriable = "value-" + BAD_RECORD_VAL_RETRIABLE; + if (badValRetriable.equals(record.value()) && shouldFail) { + shouldFail = false; + throw new RetriableException("Error when value='" + badValRetriable + + "'. A reattempt with this record will succeed."); + } + String badVal = "value-" + BAD_RECORD_VAL; + if (badVal.equals(record.value())) { + throw new RetriableException("Error when value='" + badVal + "'"); + } + return record; + } + + @Override + public ConfigDef config() { + return CONFIG_DEF; + } + + @Override + public void close() { + } + + @Override + public void configure(Map configs) { + } + } +} diff --git a/connect/runtime/src/test/java/org/apache/kafka/connect/integration/ExampleConnectIntegrationTest.java b/connect/runtime/src/test/java/org/apache/kafka/connect/integration/ExampleConnectIntegrationTest.java new file mode 100644 index 0000000000000..0648e9ff59ac3 --- /dev/null +++ b/connect/runtime/src/test/java/org/apache/kafka/connect/integration/ExampleConnectIntegrationTest.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.connect.integration; + +import org.apache.kafka.connect.runtime.rest.entities.ConnectorStateInfo; +import org.apache.kafka.connect.storage.StringConverter; +import org.apache.kafka.connect.util.clusters.EmbeddedConnectCluster; +import org.apache.kafka.test.IntegrationTest; +import org.junit.After; +import org.junit.Before; +import org.junit.Test; +import org.junit.experimental.categories.Category; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.IOException; +import java.util.HashMap; +import java.util.Map; +import java.util.Properties; + +import static org.apache.kafka.connect.runtime.ConnectorConfig.CONNECTOR_CLASS_CONFIG; +import static org.apache.kafka.connect.runtime.ConnectorConfig.KEY_CONVERTER_CLASS_CONFIG; +import static org.apache.kafka.connect.runtime.ConnectorConfig.TASKS_MAX_CONFIG; +import static org.apache.kafka.connect.runtime.ConnectorConfig.VALUE_CONVERTER_CLASS_CONFIG; +import static org.apache.kafka.connect.runtime.SinkConnectorConfig.TOPICS_CONFIG; +import static org.apache.kafka.connect.runtime.WorkerConfig.OFFSET_COMMIT_INTERVAL_MS_CONFIG; +import static org.apache.kafka.test.TestUtils.waitForCondition; +import static org.junit.Assert.assertEquals; + +/** + * An example integration test that demonstrates how to setup an integration test for Connect. + *

+ * The following test configures and executes up a sink connector pipeline in a worker, produces messages into + * the source topic-partitions, and demonstrates how to check the overall behavior of the pipeline. + */ +@Category(IntegrationTest.class) +public class ExampleConnectIntegrationTest { + + private static final Logger log = LoggerFactory.getLogger(ExampleConnectIntegrationTest.class); + + private static final int NUM_RECORDS_PRODUCED = 2000; + private static final int NUM_TOPIC_PARTITIONS = 3; + private static final int CONSUME_MAX_DURATION_MS = 5000; + private static final int CONNECTOR_SETUP_DURATION_MS = 15000; + private static final int NUM_TASKS = 3; + private static final String CONNECTOR_NAME = "simple-conn"; + + private EmbeddedConnectCluster connect; + private ConnectorHandle connectorHandle; + + @Before + public void setup() throws IOException { + // setup Connect worker properties + Map exampleWorkerProps = new HashMap<>(); + exampleWorkerProps.put(OFFSET_COMMIT_INTERVAL_MS_CONFIG, "30000"); + + // setup Kafka broker properties + Properties exampleBrokerProps = new Properties(); + exampleBrokerProps.put("auto.create.topics.enable", "false"); + + // build a Connect cluster backed by Kakfa and Zk + connect = new EmbeddedConnectCluster.Builder() + .name("example-cluster") + .numWorkers(3) + .numBrokers(1) + .workerProps(exampleWorkerProps) + .brokerProps(exampleBrokerProps) + .build(); + + // start the clusters + connect.start(); + + // get a handle to the connector + connectorHandle = RuntimeHandles.get().connectorHandle(CONNECTOR_NAME); + } + + @After + public void close() { + // delete connector handle + RuntimeHandles.get().deleteConnector(CONNECTOR_NAME); + + // stop all Connect, Kakfa and Zk threads. + connect.stop(); + } + + /** + * Simple test case to configure and execute an embedded Connect cluster. The test will produce and consume + * records, and start up a sink connector which will consume these records. + */ + @Test + public void testProduceConsumeConnector() throws Exception { + // create test topic + connect.kafka().createTopic("test-topic", NUM_TOPIC_PARTITIONS); + + // setup up props for the sink connector + Map props = new HashMap<>(); + props.put(CONNECTOR_CLASS_CONFIG, "MonitorableSink"); + props.put(TASKS_MAX_CONFIG, String.valueOf(NUM_TASKS)); + props.put(TOPICS_CONFIG, "test-topic"); + props.put(KEY_CONVERTER_CLASS_CONFIG, StringConverter.class.getName()); + props.put(VALUE_CONVERTER_CLASS_CONFIG, StringConverter.class.getName()); + + // expect all records to be consumed by the connector + connectorHandle.expectedRecords(NUM_RECORDS_PRODUCED); + + // start a sink connector + connect.configureConnector(CONNECTOR_NAME, props); + + waitForCondition(this::checkForPartitionAssignment, + CONNECTOR_SETUP_DURATION_MS, + "Connector tasks were not assigned a partition each."); + + // produce some messages into source topic partitions + for (int i = 0; i < NUM_RECORDS_PRODUCED; i++) { + connect.kafka().produce("test-topic", i % NUM_TOPIC_PARTITIONS, "key", "simple-message-value-" + i); + } + + // consume all records from the source topic or fail, to ensure that they were correctly produced. + assertEquals("Unexpected number of records consumed", NUM_RECORDS_PRODUCED, + connect.kafka().consume(NUM_RECORDS_PRODUCED, CONSUME_MAX_DURATION_MS, "test-topic").count()); + + // wait for the connector tasks to consume all records. + connectorHandle.awaitRecords(CONSUME_MAX_DURATION_MS); + + // delete connector + connect.deleteConnector(CONNECTOR_NAME); + } + + /** + * Check if a partition was assigned to each task. This method swallows exceptions since it is invoked from a + * {@link org.apache.kafka.test.TestUtils#waitForCondition} that will throw an error if this method continued + * to return false after the specified duration has elapsed. + * + * @return true if each task was assigned a partition each, false if this was not true or an error occurred when + * executing this operation. + */ + private boolean checkForPartitionAssignment() { + try { + ConnectorStateInfo info = connect.connectorStatus(CONNECTOR_NAME); + return info != null && info.tasks().size() == NUM_TASKS + && connectorHandle.tasks().stream().allMatch(th -> th.partitionsAssigned() == 1); + } catch (Exception e) { + // Log the exception and return that the partitions were not assigned + log.error("Could not check connector state info.", e); + return false; + } + } +} diff --git a/connect/runtime/src/test/java/org/apache/kafka/connect/integration/MonitorableSinkConnector.java b/connect/runtime/src/test/java/org/apache/kafka/connect/integration/MonitorableSinkConnector.java new file mode 100644 index 0000000000000..23a8d99e84edc --- /dev/null +++ b/connect/runtime/src/test/java/org/apache/kafka/connect/integration/MonitorableSinkConnector.java @@ -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. + */ +package org.apache.kafka.connect.integration; + +import org.apache.kafka.common.TopicPartition; +import org.apache.kafka.common.config.ConfigDef; +import org.apache.kafka.connect.connector.Task; +import org.apache.kafka.connect.runtime.TestSinkConnector; +import org.apache.kafka.connect.sink.SinkRecord; +import org.apache.kafka.connect.sink.SinkTask; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.util.ArrayList; +import java.util.Collection; +import java.util.HashMap; +import java.util.List; +import java.util.Map; + +/** + * A connector to be used in integration tests. This class provides methods to find task instances + * which are initiated by the embedded connector, and wait for them to consume a desired number of + * messages. + */ +public class MonitorableSinkConnector extends TestSinkConnector { + + private static final Logger log = LoggerFactory.getLogger(MonitorableSinkConnector.class); + + private String connectorName; + + @Override + public void start(Map props) { + connectorName = props.get("name"); + log.info("Starting connector {}", props.get("name")); + } + + @Override + public Class taskClass() { + return MonitorableSinkTask.class; + } + + @Override + public List> taskConfigs(int maxTasks) { + List> configs = new ArrayList<>(); + for (int i = 0; i < maxTasks; i++) { + Map config = new HashMap<>(); + config.put("connector.name", connectorName); + config.put("task.id", connectorName + "-" + i); + configs.add(config); + } + return configs; + } + + @Override + public void stop() { + } + + @Override + public ConfigDef config() { + return new ConfigDef(); + } + + public static class MonitorableSinkTask extends SinkTask { + + private String connectorName; + private String taskId; + private TaskHandle taskHandle; + + @Override + public String version() { + return "unknown"; + } + + @Override + public void start(Map props) { + taskId = props.get("task.id"); + connectorName = props.get("connector.name"); + taskHandle = RuntimeHandles.get().connectorHandle(connectorName).taskHandle(taskId); + log.debug("Starting task {}", taskId); + } + + @Override + public void open(Collection partitions) { + log.debug("Opening {} partitions", partitions.size()); + super.open(partitions); + taskHandle.partitionsAssigned(partitions.size()); + } + + @Override + public void put(Collection records) { + for (SinkRecord rec : records) { + taskHandle.record(); + log.trace("Task {} obtained record (key='{}' value='{}')", taskId, rec.key(), rec.value()); + } + } + + @Override + public void stop() { + } + } +} diff --git a/connect/runtime/src/test/java/org/apache/kafka/connect/integration/RestExtensionIntegrationTest.java b/connect/runtime/src/test/java/org/apache/kafka/connect/integration/RestExtensionIntegrationTest.java new file mode 100644 index 0000000000000..24b3d8b49e408 --- /dev/null +++ b/connect/runtime/src/test/java/org/apache/kafka/connect/integration/RestExtensionIntegrationTest.java @@ -0,0 +1,208 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.kafka.connect.integration; + +import org.apache.kafka.connect.errors.NotFoundException; +import org.apache.kafka.connect.health.ConnectClusterState; +import org.apache.kafka.connect.health.ConnectorHealth; +import org.apache.kafka.connect.health.ConnectorState; +import org.apache.kafka.connect.health.ConnectorType; +import org.apache.kafka.connect.health.TaskState; +import org.apache.kafka.connect.rest.ConnectRestExtension; +import org.apache.kafka.connect.rest.ConnectRestExtensionContext; +import org.apache.kafka.connect.runtime.rest.errors.ConnectRestException; +import org.apache.kafka.connect.util.clusters.EmbeddedConnectCluster; +import org.apache.kafka.test.IntegrationTest; +import org.junit.After; +import org.junit.Test; +import org.junit.experimental.categories.Category; + +import javax.ws.rs.GET; +import javax.ws.rs.Path; +import java.io.IOException; +import java.net.URI; +import java.net.URISyntaxException; +import java.util.Collections; +import java.util.HashMap; +import java.util.Map; +import java.util.concurrent.TimeUnit; + +import static org.apache.kafka.connect.runtime.ConnectorConfig.CONNECTOR_CLASS_CONFIG; +import static org.apache.kafka.connect.runtime.ConnectorConfig.NAME_CONFIG; +import static org.apache.kafka.connect.runtime.ConnectorConfig.TASKS_MAX_CONFIG; +import static org.apache.kafka.connect.runtime.SinkConnectorConfig.TOPICS_CONFIG; +import static org.apache.kafka.connect.runtime.WorkerConfig.REST_EXTENSION_CLASSES_CONFIG; +import static org.apache.kafka.test.TestUtils.waitForCondition; +import static org.junit.Assert.assertEquals; + +/** + * A simple integration test to ensure that REST extensions are registered correctly. + */ +@Category(IntegrationTest.class) +public class RestExtensionIntegrationTest { + + private static final long REST_EXTENSION_REGISTRATION_TIMEOUT_MS = TimeUnit.MINUTES.toMillis(1); + private static final long CONNECTOR_HEALTH_AND_CONFIG_TIMEOUT_MS = TimeUnit.MINUTES.toMillis(1); + + private EmbeddedConnectCluster connect; + + @Test + public void testRestExtensionApi() throws IOException, InterruptedException, URISyntaxException { + // setup Connect worker properties + Map workerProps = new HashMap<>(); + workerProps.put(REST_EXTENSION_CLASSES_CONFIG, IntegrationTestRestExtension.class.getName()); + + // build a Connect cluster backed by Kafka and Zk + connect = new EmbeddedConnectCluster.Builder() + .name("connect-cluster") + .numWorkers(1) + .numBrokers(1) + .workerProps(workerProps) + .build(); + + // start the clusters + connect.start(); + + waitForCondition( + this::extensionIsRegistered, + REST_EXTENSION_REGISTRATION_TIMEOUT_MS, + "REST extension was never registered" + ); + + final String connectorName = "test-conn"; + try { + // setup up props for the connector + Map connectorProps = new HashMap<>(); + connectorProps.put(CONNECTOR_CLASS_CONFIG, MonitorableSinkConnector.class.getSimpleName()); + connectorProps.put(TASKS_MAX_CONFIG, String.valueOf(1)); + connectorProps.put(TOPICS_CONFIG, "test-topic"); + + // start a connector + connect.configureConnector(connectorName, connectorProps); + + URI workerUrl = new URI(connect.endpointForResource("")); + String workerId = workerUrl.getHost() + ":" + workerUrl.getPort(); + ConnectorHealth expectedHealth = new ConnectorHealth( + connectorName, + new ConnectorState( + "RUNNING", + workerId, + null + ), + Collections.singletonMap( + 0, + new TaskState(0, "RUNNING", workerId, null) + ), + ConnectorType.SINK + ); + + connectorProps.put(NAME_CONFIG, connectorName); + + // Test the REST extension API; specifically, that the connector's health is available + // to the REST extension we registered and that they contain expected values + waitForCondition( + () -> verifyConnectorHealth(connectorName, expectedHealth), + CONNECTOR_HEALTH_AND_CONFIG_TIMEOUT_MS, + "Connector health and/or config was never accessible by the REST extension" + ); + } finally { + RuntimeHandles.get().deleteConnector(connectorName); + } + } + + @After + public void close() { + // stop all Connect, Kafka and Zk threads. + connect.stop(); + IntegrationTestRestExtension.instance = null; + } + + private boolean extensionIsRegistered() { + try { + String extensionUrl = connect.endpointForResource("integration-test-rest-extension/registered"); + return "true".equals(connect.executeGet(extensionUrl)); + } catch (ConnectRestException | IOException e) { + return false; + } + } + + private boolean verifyConnectorHealth( + String connectorName, + ConnectorHealth expectedHealth + ) { + ConnectClusterState clusterState = + IntegrationTestRestExtension.instance.restPluginContext.clusterState(); + + ConnectorHealth actualHealth; + try { + actualHealth = clusterState.connectorHealth(connectorName); + } catch (NotFoundException e) { + // Happens if the connector hasn't been started yet by the worker + return false; + } + + if (actualHealth.tasksState().isEmpty()) { + // Happens if the task has been started but its status has not yet been picked up from + // the status topic by the worker. + return false; + } + assertEquals(expectedHealth, actualHealth); + + return true; + } + + public static class IntegrationTestRestExtension implements ConnectRestExtension { + private static IntegrationTestRestExtension instance; + + public ConnectRestExtensionContext restPluginContext; + + @Override + public void register(ConnectRestExtensionContext restPluginContext) { + instance = this; + this.restPluginContext = restPluginContext; + // Immediately request a list of connectors to confirm that the context and its fields + // has been fully initialized and there is no risk of deadlock + restPluginContext.clusterState().connectors(); + // Install a new REST resource that can be used to confirm that the extension has been + // successfully registered + restPluginContext.configurable().register(new IntegrationTestRestExtensionResource()); + } + + @Override + public void close() { + } + + @Override + public void configure(Map configs) { + } + + @Override + public String version() { + return "test"; + } + + @Path("integration-test-rest-extension") + public static class IntegrationTestRestExtensionResource { + + @GET + @Path("/registered") + public boolean isRegistered() { + return true; + } + } + } +} diff --git a/connect/runtime/src/test/java/org/apache/kafka/connect/integration/RuntimeHandles.java b/connect/runtime/src/test/java/org/apache/kafka/connect/integration/RuntimeHandles.java new file mode 100644 index 0000000000000..c9900f3a7fb62 --- /dev/null +++ b/connect/runtime/src/test/java/org/apache/kafka/connect/integration/RuntimeHandles.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.connect.integration; + +import java.util.Map; +import java.util.concurrent.ConcurrentHashMap; + +/** + * A singleton class which provides a shared class for {@link ConnectorHandle}s and {@link TaskHandle}s that are + * required for integration tests. + */ +public class RuntimeHandles { + + private static final RuntimeHandles INSTANCE = new RuntimeHandles(); + + private final Map connectorHandles = new ConcurrentHashMap<>(); + + private RuntimeHandles() { + } + + /** + * @return the shared {@link RuntimeHandles} instance. + */ + public static RuntimeHandles get() { + return INSTANCE; + } + + /** + * Get or create a connector handle for a given connector name. The connector need not be running at the time + * this method is called. Once the connector is created, it will bind to this handle. Binding happens with the + * connectorName. + * + * @param connectorName the name of the connector + * @return a non-null {@link ConnectorHandle} + */ + public ConnectorHandle connectorHandle(String connectorName) { + return connectorHandles.computeIfAbsent(connectorName, k -> new ConnectorHandle(connectorName)); + } + + /** + * Delete the connector handle for this connector name. + * + * @param connectorName name of the connector + */ + public void deleteConnector(String connectorName) { + connectorHandles.remove(connectorName); + } + +} diff --git a/connect/runtime/src/test/java/org/apache/kafka/connect/integration/TaskHandle.java b/connect/runtime/src/test/java/org/apache/kafka/connect/integration/TaskHandle.java new file mode 100644 index 0000000000000..de3d9240d1be7 --- /dev/null +++ b/connect/runtime/src/test/java/org/apache/kafka/connect/integration/TaskHandle.java @@ -0,0 +1,111 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.kafka.connect.integration; + +import org.apache.kafka.connect.errors.DataException; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.util.concurrent.CountDownLatch; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicInteger; + +/** + * A handle to an executing task in a worker. Use this class to record progress, for example: number of records seen + * by the task using so far, or waiting for partitions to be assigned to the task. + */ +public class TaskHandle { + + private static final Logger log = LoggerFactory.getLogger(TaskHandle.class); + + private final String taskId; + private final ConnectorHandle connectorHandle; + private final AtomicInteger partitionsAssigned = new AtomicInteger(0); + + private CountDownLatch recordsRemainingLatch; + private int expectedRecords = -1; + + public TaskHandle(ConnectorHandle connectorHandle, String taskId) { + log.info("Created task {} for connector {}", taskId, connectorHandle); + this.taskId = taskId; + this.connectorHandle = connectorHandle; + } + + /** + * Record a message arrival at the task. + */ + public void record() { + if (recordsRemainingLatch != null) { + recordsRemainingLatch.countDown(); + } + connectorHandle.record(); + } + + /** + * Set the number of expected records for this task. + * + * @param expectedRecords number of records + */ + public void expectedRecords(int expectedRecords) { + this.expectedRecords = expectedRecords; + this.recordsRemainingLatch = new CountDownLatch(expectedRecords); + } + + /** + * Set the number of partitions assigned to this task. + * + * @param numPartitions number of partitions + */ + public void partitionsAssigned(int numPartitions) { + partitionsAssigned.set(numPartitions); + } + + /** + * @return the number of topic partitions assigned to this task. + */ + public int partitionsAssigned() { + return partitionsAssigned.get(); + } + + /** + * Wait for this task to receive the expected number of records. + * + * @param consumeMaxDurationMs max duration to wait for records + * @throws InterruptedException if another threads interrupts this one while waiting for records + */ + public void awaitRecords(int consumeMaxDurationMs) throws InterruptedException { + if (recordsRemainingLatch == null) { + throw new IllegalStateException("Illegal state encountered. expectedRecords() was not set for this task?"); + } + if (!recordsRemainingLatch.await(consumeMaxDurationMs, TimeUnit.MILLISECONDS)) { + String msg = String.format("Insufficient records seen by task %s in %d millis. Records expected=%d, actual=%d", + taskId, + consumeMaxDurationMs, + expectedRecords, + expectedRecords - recordsRemainingLatch.getCount()); + throw new DataException(msg); + } + log.debug("Task {} saw {} records, expected {} records", taskId, expectedRecords - recordsRemainingLatch.getCount(), expectedRecords); + } + + @Override + public String toString() { + return "Handle{" + + "taskId='" + taskId + '\'' + + '}'; + } +} diff --git a/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/AbstractHerderTest.java b/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/AbstractHerderTest.java index 5728465095a45..c85764eb42d65 100644 --- a/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/AbstractHerderTest.java +++ b/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/AbstractHerderTest.java @@ -18,14 +18,18 @@ import org.apache.kafka.common.config.ConfigDef; import org.apache.kafka.common.config.ConfigException; +import org.apache.kafka.common.config.ConfigTransformer; import org.apache.kafka.connect.connector.ConnectRecord; import org.apache.kafka.connect.connector.Connector; +import org.apache.kafka.connect.runtime.distributed.ClusterConfigState; import org.apache.kafka.connect.runtime.isolation.PluginDesc; import org.apache.kafka.connect.runtime.isolation.Plugins; import org.apache.kafka.connect.runtime.rest.entities.ConfigInfos; import org.apache.kafka.connect.runtime.rest.entities.ConnectorStateInfo; import org.apache.kafka.connect.runtime.rest.entities.ConnectorType; import org.apache.kafka.connect.runtime.rest.errors.BadRequestException; +import org.apache.kafka.connect.source.SourceConnector; +import org.apache.kafka.connect.source.SourceTask; import org.apache.kafka.connect.storage.ConfigBackingStore; import org.apache.kafka.connect.storage.StatusBackingStore; import org.apache.kafka.connect.transforms.Transformation; @@ -40,6 +44,7 @@ import org.powermock.core.classloader.annotations.PrepareForTest; import org.powermock.modules.junit4.PowerMockRunner; +import java.util.ArrayList; import java.util.Arrays; import java.util.Collections; import java.util.HashMap; @@ -48,6 +53,7 @@ import java.util.Map; import java.util.Set; +import static org.apache.kafka.connect.runtime.AbstractHerder.keysWithVariableValues; import static org.powermock.api.easymock.PowerMock.verifyAll; import static org.powermock.api.easymock.PowerMock.replayAll; import static org.easymock.EasyMock.strictMock; @@ -61,12 +67,60 @@ @PrepareForTest({AbstractHerder.class}) public class AbstractHerderTest { + private static final String CONN1 = "sourceA"; + private static final ConnectorTaskId TASK0 = new ConnectorTaskId(CONN1, 0); + private static final ConnectorTaskId TASK1 = new ConnectorTaskId(CONN1, 1); + private static final ConnectorTaskId TASK2 = new ConnectorTaskId(CONN1, 2); + private static final Integer MAX_TASKS = 3; + private static final Map CONN1_CONFIG = new HashMap<>(); + private static final String TEST_KEY = "testKey"; + private static final String TEST_KEY2 = "testKey2"; + private static final String TEST_KEY3 = "testKey3"; + private static final String TEST_VAL = "testVal"; + private static final String TEST_VAL2 = "testVal2"; + private static final String TEST_REF = "${file:/tmp/somefile.txt:somevar}"; + private static final String TEST_REF2 = "${file:/tmp/somefile2.txt:somevar2}"; + private static final String TEST_REF3 = "${file:/tmp/somefile3.txt:somevar3}"; + static { + CONN1_CONFIG.put(ConnectorConfig.NAME_CONFIG, CONN1); + CONN1_CONFIG.put(ConnectorConfig.TASKS_MAX_CONFIG, MAX_TASKS.toString()); + CONN1_CONFIG.put(SinkConnectorConfig.TOPICS_CONFIG, "foo,bar"); + CONN1_CONFIG.put(ConnectorConfig.CONNECTOR_CLASS_CONFIG, BogusSourceConnector.class.getName()); + CONN1_CONFIG.put(TEST_KEY, TEST_REF); + CONN1_CONFIG.put(TEST_KEY2, TEST_REF2); + CONN1_CONFIG.put(TEST_KEY3, TEST_REF3); + } + private static final Map TASK_CONFIG = new HashMap<>(); + static { + TASK_CONFIG.put(TaskConfig.TASK_CLASS_CONFIG, BogusSourceTask.class.getName()); + TASK_CONFIG.put(TEST_KEY, TEST_REF); + } + private static final List> TASK_CONFIGS = new ArrayList<>(); + static { + TASK_CONFIGS.add(TASK_CONFIG); + TASK_CONFIGS.add(TASK_CONFIG); + TASK_CONFIGS.add(TASK_CONFIG); + } + private static final HashMap> TASK_CONFIGS_MAP = new HashMap<>(); + static { + TASK_CONFIGS_MAP.put(TASK0, TASK_CONFIG); + TASK_CONFIGS_MAP.put(TASK1, TASK_CONFIG); + TASK_CONFIGS_MAP.put(TASK2, TASK_CONFIG); + } + private static final ClusterConfigState SNAPSHOT = new ClusterConfigState(1, Collections.singletonMap(CONN1, 3), + Collections.singletonMap(CONN1, CONN1_CONFIG), Collections.singletonMap(CONN1, TargetState.STARTED), + TASK_CONFIGS_MAP, Collections.emptySet()); + private static final ClusterConfigState SNAPSHOT_NO_TASKS = new ClusterConfigState(1, Collections.singletonMap(CONN1, 3), + Collections.singletonMap(CONN1, CONN1_CONFIG), Collections.singletonMap(CONN1, TargetState.STARTED), + Collections.emptyMap(), Collections.emptySet()); + private final String workerId = "workerId"; private final String kafkaClusterId = "I4ZmrWqfT2e-upky_4fdPA"; private final int generation = 5; private final String connector = "connector"; @MockStrict private Worker worker; + @MockStrict private WorkerConfigTransformer transformer; @MockStrict private Plugins plugins; @MockStrict private ClassLoader classLoader; @MockStrict private ConfigBackingStore configStore; @@ -247,6 +301,55 @@ public void testConfigValidationTransformsExtendResults() { verifyAll(); } + @Test + public void testReverseTransformConfigs() throws Exception { + // Construct a task config with constant values for TEST_KEY and TEST_KEY2 + Map newTaskConfig = new HashMap<>(); + newTaskConfig.put(TaskConfig.TASK_CLASS_CONFIG, BogusSourceTask.class.getName()); + newTaskConfig.put(TEST_KEY, TEST_VAL); + newTaskConfig.put(TEST_KEY2, TEST_VAL2); + List> newTaskConfigs = new ArrayList<>(); + newTaskConfigs.add(newTaskConfig); + + // The SNAPSHOT has a task config with TEST_KEY and TEST_REF + List> reverseTransformed = AbstractHerder.reverseTransform(CONN1, SNAPSHOT, newTaskConfigs); + assertEquals(TEST_REF, reverseTransformed.get(0).get(TEST_KEY)); + + // The SNAPSHOT has no task configs but does have a connector config with TEST_KEY2 and TEST_REF2 + reverseTransformed = AbstractHerder.reverseTransform(CONN1, SNAPSHOT_NO_TASKS, newTaskConfigs); + assertEquals(TEST_REF2, reverseTransformed.get(0).get(TEST_KEY2)); + + // The reverseTransformed result should not have TEST_KEY3 since newTaskConfigs does not have TEST_KEY3 + reverseTransformed = AbstractHerder.reverseTransform(CONN1, SNAPSHOT_NO_TASKS, newTaskConfigs); + assertFalse(reverseTransformed.get(0).containsKey(TEST_KEY3)); + } + + @Test + public void testConfigProviderRegex() { + testConfigProviderRegex("\"${::}\""); + testConfigProviderRegex("${::}"); + testConfigProviderRegex("\"${:/a:somevar}\""); + testConfigProviderRegex("\"${file::somevar}\""); + testConfigProviderRegex("${file:/a/b/c:}"); + testConfigProviderRegex("${file:/tmp/somefile.txt:somevar}"); + testConfigProviderRegex("\"${file:/tmp/somefile.txt:somevar}\""); + testConfigProviderRegex("plain.PlainLoginModule required username=\"${file:/tmp/somefile.txt:somevar}\""); + testConfigProviderRegex("plain.PlainLoginModule required username=${file:/tmp/somefile.txt:somevar}"); + testConfigProviderRegex("plain.PlainLoginModule required username=${file:/tmp/somefile.txt:somevar} not null"); + testConfigProviderRegex("plain.PlainLoginModule required username=${file:/tmp/somefile.txt:somevar} password=${file:/tmp/somefile.txt:othervar}"); + testConfigProviderRegex("plain.PlainLoginModule required username", false); + } + + private void testConfigProviderRegex(String rawConnConfig) { + testConfigProviderRegex(rawConnConfig, true); + } + + private void testConfigProviderRegex(String rawConnConfig, boolean expected) { + Set keys = keysWithVariableValues(Collections.singletonMap("key", rawConnConfig), ConfigTransformer.DEFAULT_PATTERN); + boolean actual = keys != null && !keys.isEmpty() && keys.contains("key"); + assertEquals(String.format("%s should have matched regex", rawConnConfig), expected, actual); + } + private AbstractHerder createConfigValidationHerder(Class connectorClass) { @@ -261,6 +364,9 @@ private AbstractHerder createConfigValidationHerder(Class c EasyMock.expect(herder.generation()).andStubReturn(generation); // Call to validateConnectorConfig + EasyMock.expect(worker.configTransformer()).andReturn(transformer).times(2); + final Capture> configCapture = EasyMock.newCapture(); + EasyMock.expect(transformer.transform(EasyMock.capture(configCapture))).andAnswer(configCapture::getValue); EasyMock.expect(worker.getPlugins()).andStubReturn(plugins); final Connector connector; try { @@ -295,4 +401,11 @@ public void close() { } } + + // We need to use a real class here due to some issue with mocking java.lang.Class + private abstract class BogusSourceConnector extends SourceConnector { + } + + private abstract class BogusSourceTask extends SourceTask { + } } diff --git a/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/ErrorHandlingTaskTest.java b/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/ErrorHandlingTaskTest.java index e931642afcc75..3a7d5a0372eb7 100644 --- a/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/ErrorHandlingTaskTest.java +++ b/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/ErrorHandlingTaskTest.java @@ -46,7 +46,7 @@ import org.apache.kafka.connect.source.SourceTask; import org.apache.kafka.connect.storage.Converter; import org.apache.kafka.connect.storage.HeaderConverter; -import org.apache.kafka.connect.storage.OffsetStorageReader; +import org.apache.kafka.connect.storage.OffsetStorageReaderImpl; import org.apache.kafka.connect.storage.OffsetStorageWriter; import org.apache.kafka.connect.transforms.Transformation; import org.apache.kafka.connect.transforms.util.SimpleConfig; @@ -126,7 +126,7 @@ public class ErrorHandlingTaskTest { private KafkaProducer producer; @Mock - OffsetStorageReader offsetReader; + OffsetStorageReaderImpl offsetReader; @Mock OffsetStorageWriter offsetWriter; @@ -166,8 +166,7 @@ public void testErrorHandlingInSinkTasks() throws Exception { Map reportProps = new HashMap<>(); reportProps.put(ConnectorConfig.ERRORS_LOG_ENABLE_CONFIG, "true"); reportProps.put(ConnectorConfig.ERRORS_LOG_INCLUDE_MESSAGES_CONFIG, "true"); - LogReporter reporter = new LogReporter(taskId, connConfig(reportProps)); - reporter.metrics(errorHandlingMetrics); + LogReporter reporter = new LogReporter(taskId, connConfig(reportProps), errorHandlingMetrics); RetryWithToleranceOperator retryWithToleranceOperator = operator(); retryWithToleranceOperator.metrics(errorHandlingMetrics); @@ -218,8 +217,7 @@ public void testErrorHandlingInSourceTasks() throws Exception { Map reportProps = new HashMap<>(); reportProps.put(ConnectorConfig.ERRORS_LOG_ENABLE_CONFIG, "true"); reportProps.put(ConnectorConfig.ERRORS_LOG_INCLUDE_MESSAGES_CONFIG, "true"); - LogReporter reporter = new LogReporter(taskId, connConfig(reportProps)); - reporter.metrics(errorHandlingMetrics); + LogReporter reporter = new LogReporter(taskId, connConfig(reportProps), errorHandlingMetrics); RetryWithToleranceOperator retryWithToleranceOperator = operator(); retryWithToleranceOperator.metrics(errorHandlingMetrics); @@ -283,8 +281,7 @@ public void testErrorHandlingInSourceTasksWthBadConverter() throws Exception { Map reportProps = new HashMap<>(); reportProps.put(ConnectorConfig.ERRORS_LOG_ENABLE_CONFIG, "true"); reportProps.put(ConnectorConfig.ERRORS_LOG_INCLUDE_MESSAGES_CONFIG, "true"); - LogReporter reporter = new LogReporter(taskId, connConfig(reportProps)); - reporter.metrics(errorHandlingMetrics); + LogReporter reporter = new LogReporter(taskId, connConfig(reportProps), errorHandlingMetrics); RetryWithToleranceOperator retryWithToleranceOperator = operator(); retryWithToleranceOperator.metrics(errorHandlingMetrics); diff --git a/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/WorkerConfigTransformerTest.java b/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/WorkerConfigTransformerTest.java index 89bba09b0da26..034bd5130719b 100644 --- a/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/WorkerConfigTransformerTest.java +++ b/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/WorkerConfigTransformerTest.java @@ -18,7 +18,7 @@ import org.apache.kafka.common.config.ConfigChangeCallback; import org.apache.kafka.common.config.ConfigData; -import org.apache.kafka.common.config.ConfigProvider; +import org.apache.kafka.common.config.provider.ConfigProvider; import org.easymock.EasyMock; import org.junit.Before; import org.junit.Test; @@ -28,10 +28,14 @@ import org.powermock.modules.junit4.PowerMockRunner; import java.util.Collections; +import java.util.HashMap; import java.util.Map; import java.util.Set; +import static org.apache.kafka.connect.runtime.ConnectorConfig.CONFIG_RELOAD_ACTION_CONFIG; +import static org.apache.kafka.connect.runtime.ConnectorConfig.CONFIG_RELOAD_ACTION_NONE; import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertNull; import static org.powermock.api.easymock.PowerMock.replayAll; @RunWith(PowerMockRunner.class) @@ -68,18 +72,18 @@ public void testReplaceVariable() throws Exception { @Test public void testReplaceVariableWithTTL() throws Exception { EasyMock.expect(worker.herder()).andReturn(herder); - EasyMock.expect(herder.connectorConfigReloadAction(MY_CONNECTOR)).andReturn(Herder.ConfigReloadAction.NONE); replayAll(); - Map result = configTransformer.transform(MY_CONNECTOR, Collections.singletonMap(MY_KEY, "${test:testPath:testKeyWithTTL}")); - assertEquals(TEST_RESULT_WITH_TTL, result.get(MY_KEY)); + Map props = new HashMap<>(); + props.put(MY_KEY, "${test:testPath:testKeyWithTTL}"); + props.put(CONFIG_RELOAD_ACTION_CONFIG, CONFIG_RELOAD_ACTION_NONE); + Map result = configTransformer.transform(MY_CONNECTOR, props); } @Test public void testReplaceVariableWithTTLAndScheduleRestart() throws Exception { EasyMock.expect(worker.herder()).andReturn(herder); - EasyMock.expect(herder.connectorConfigReloadAction(MY_CONNECTOR)).andReturn(Herder.ConfigReloadAction.RESTART); EasyMock.expect(herder.restartConnector(1L, MY_CONNECTOR, null)).andReturn(requestId); replayAll(); @@ -91,11 +95,9 @@ public void testReplaceVariableWithTTLAndScheduleRestart() throws Exception { @Test public void testReplaceVariableWithTTLFirstCancelThenScheduleRestart() throws Exception { EasyMock.expect(worker.herder()).andReturn(herder); - EasyMock.expect(herder.connectorConfigReloadAction(MY_CONNECTOR)).andReturn(Herder.ConfigReloadAction.RESTART); EasyMock.expect(herder.restartConnector(1L, MY_CONNECTOR, null)).andReturn(requestId); EasyMock.expect(worker.herder()).andReturn(herder); - EasyMock.expect(herder.connectorConfigReloadAction(MY_CONNECTOR)).andReturn(Herder.ConfigReloadAction.RESTART); EasyMock.expectLastCall(); requestId.cancel(); EasyMock.expectLastCall(); @@ -110,6 +112,11 @@ public void testReplaceVariableWithTTLFirstCancelThenScheduleRestart() throws Ex assertEquals(TEST_RESULT_WITH_LONGER_TTL, result.get(MY_KEY)); } + @Test + public void testTransformNullConfiguration() { + assertNull(configTransformer.transform(MY_CONNECTOR, null)); + } + public static class TestConfigProvider implements ConfigProvider { public void configure(Map configs) { diff --git a/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/WorkerSourceTaskTest.java b/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/WorkerSourceTaskTest.java index db73a8e091483..e0df33e56c7b5 100644 --- a/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/WorkerSourceTaskTest.java +++ b/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/WorkerSourceTaskTest.java @@ -21,9 +21,11 @@ import org.apache.kafka.clients.producer.RecordMetadata; import org.apache.kafka.common.MetricName; import org.apache.kafka.common.TopicPartition; +import org.apache.kafka.common.errors.TopicAuthorizationException; import org.apache.kafka.common.record.InvalidRecordException; import org.apache.kafka.common.utils.Time; import org.apache.kafka.connect.data.Schema; +import org.apache.kafka.connect.errors.ConnectException; import org.apache.kafka.connect.runtime.ConnectMetrics.MetricGroup; import org.apache.kafka.connect.runtime.WorkerSourceTask.SourceTaskMetricsGroup; import org.apache.kafka.connect.runtime.distributed.ClusterConfigState; @@ -33,9 +35,9 @@ import org.apache.kafka.connect.source.SourceRecord; import org.apache.kafka.connect.source.SourceTask; import org.apache.kafka.connect.source.SourceTaskContext; +import org.apache.kafka.connect.storage.CloseableOffsetStorageReader; import org.apache.kafka.connect.storage.Converter; import org.apache.kafka.connect.storage.HeaderConverter; -import org.apache.kafka.connect.storage.OffsetStorageReader; import org.apache.kafka.connect.storage.OffsetStorageWriter; import org.apache.kafka.connect.util.Callback; import org.apache.kafka.connect.util.ConnectorTaskId; @@ -74,6 +76,7 @@ import static org.junit.Assert.assertTrue; @PowerMockIgnore({"javax.management.*", + "org.apache.log4j.*", "org.apache.kafka.connect.runtime.isolation.*"}) @RunWith(PowerMockRunner.class) public class WorkerSourceTaskTest extends ThreadedTest { @@ -103,7 +106,7 @@ public class WorkerSourceTaskTest extends ThreadedTest { @Mock private HeaderConverter headerConverter; @Mock private TransformationChain transformationChain; @Mock private KafkaProducer producer; - @Mock private OffsetStorageReader offsetReader; + @Mock private CloseableOffsetStorageReader offsetReader; @Mock private OffsetStorageWriter offsetWriter; @Mock private ClusterConfigState clusterConfigState; private WorkerSourceTask workerTask; @@ -336,6 +339,51 @@ public List answer() throws Throwable { PowerMock.verifyAll(); } + @Test + public void testPollReturnsNoRecords() throws Exception { + // Test that the task handles an empty list of records + createWorkerTask(); + + sourceTask.initialize(EasyMock.anyObject(SourceTaskContext.class)); + EasyMock.expectLastCall(); + sourceTask.start(TASK_PROPS); + EasyMock.expectLastCall(); + statusListener.onStartup(taskId); + EasyMock.expectLastCall(); + + // We'll wait for some data, then trigger a flush + final CountDownLatch pollLatch = expectEmptyPolls(1, new AtomicInteger()); + expectOffsetFlush(true); + + sourceTask.stop(); + EasyMock.expectLastCall(); + expectOffsetFlush(true); + + statusListener.onShutdown(taskId); + EasyMock.expectLastCall(); + + producer.close(EasyMock.anyInt(), EasyMock.anyObject(TimeUnit.class)); + EasyMock.expectLastCall(); + + transformationChain.close(); + EasyMock.expectLastCall(); + + PowerMock.replayAll(); + + workerTask.initialize(TASK_CONFIG); + Future taskFuture = executor.submit(workerTask); + + assertTrue(awaitLatch(pollLatch)); + assertTrue(workerTask.commitOffsets()); + workerTask.stop(); + assertTrue(workerTask.awaitStop(1000)); + + taskFuture.get(); + assertPollMetrics(0); + + PowerMock.verifyAll(); + } + @Test public void testCommit() throws Exception { // Test that the task commits properly when prompted @@ -541,6 +589,21 @@ public void testSendRecordsRetries() throws Exception { PowerMock.verifyAll(); } + @Test(expected = ConnectException.class) + public void testSendRecordsProducerCallbackFail() throws Exception { + createWorkerTask(); + + SourceRecord record1 = new SourceRecord(PARTITION, OFFSET, "topic", 1, KEY_SCHEMA, KEY, RECORD_SCHEMA, RECORD); + SourceRecord record2 = new SourceRecord(PARTITION, OFFSET, "topic", 2, KEY_SCHEMA, KEY, RECORD_SCHEMA, RECORD); + + expectSendRecordProducerCallbackFail(); + + PowerMock.replayAll(); + + Whitebox.setInternalState(workerTask, "toSend", Arrays.asList(record1, record2)); + Whitebox.invokeMethod(workerTask, "sendRecords"); + } + @Test public void testSendRecordsTaskCommitRecordFail() throws Exception { createWorkerTask(); @@ -618,6 +681,20 @@ public Object answer() throws Throwable { PowerMock.verifyAll(); } + @Test + public void testCancel() { + createWorkerTask(); + + offsetReader.close(); + PowerMock.expectLastCall(); + + PowerMock.replayAll(); + + workerTask.cancel(); + + PowerMock.verifyAll(); + } + @Test public void testMetricsGroup() { SourceTaskMetricsGroup group = new SourceTaskMetricsGroup(taskId, metrics); @@ -664,6 +741,24 @@ public void testMetricsGroup() { assertEquals(1800.0, metrics.currentMetricValueAsDouble(group1.metricGroup(), "source-record-active-count"), 0.001d); } + private CountDownLatch expectEmptyPolls(int minimum, final AtomicInteger count) throws InterruptedException { + final CountDownLatch latch = new CountDownLatch(minimum); + // 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 { + count.incrementAndGet(); + latch.countDown(); + Thread.sleep(10); + return Collections.emptyList(); + } + }); + return latch; + } + private CountDownLatch expectPolls(int minimum, final AtomicInteger count) throws InterruptedException { final CountDownLatch latch = new CountDownLatch(minimum); // Note that we stub these to allow any number of calls because the thread will continue to @@ -710,16 +805,24 @@ private Capture> expectSendRecordOnce(boolean isR return expectSendRecordTaskCommitRecordSucceed(false, isRetry); } + private Capture> expectSendRecordProducerCallbackFail() throws InterruptedException { + return expectSendRecord(false, false, false, false); + } + private Capture> expectSendRecordTaskCommitRecordSucceed(boolean anyTimes, boolean isRetry) throws InterruptedException { - return expectSendRecord(anyTimes, isRetry, true); + return expectSendRecord(anyTimes, isRetry, true, true); } private Capture> expectSendRecordTaskCommitRecordFail(boolean anyTimes, boolean isRetry) throws InterruptedException { - return expectSendRecord(anyTimes, isRetry, false); + return expectSendRecord(anyTimes, isRetry, true, false); } - @SuppressWarnings("unchecked") - private Capture> expectSendRecord(boolean anyTimes, boolean isRetry, boolean succeed) throws InterruptedException { + private Capture> expectSendRecord( + boolean anyTimes, + boolean isRetry, + boolean sendSuccess, + boolean commitSuccess + ) throws InterruptedException { expectConvertKeyValue(anyTimes); expectApplyTransformationChain(anyTimes); @@ -736,15 +839,19 @@ private Capture> expectSendRecord(boolean anyTime // 2. Converted data passed to the producer, which will need callbacks invoked for flush to work IExpectationSetters> expect = EasyMock.expect( - producer.send(EasyMock.capture(sent), - EasyMock.capture(producerCallbacks))); + producer.send(EasyMock.capture(sent), + EasyMock.capture(producerCallbacks))); IAnswer> expectResponse = 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, - 0L, 0L, 0, 0), null); + if (sendSuccess) { + cb.onCompletion(new RecordMetadata(new TopicPartition("foo", 0), 0, 0, + 0L, 0L, 0, 0), null); + } else { + cb.onCompletion(null, new TopicAuthorizationException("foo")); + } } producerCallbacks.reset(); } @@ -756,8 +863,10 @@ public Future answer() throws Throwable { else expect.andAnswer(expectResponse); - // 3. As a result of a successful producer send callback, we'll notify the source task of the record commit - expectTaskCommitRecord(anyTimes, succeed); + if (sendSuccess) { + // 3. As a result of a successful producer send callback, we'll notify the source task of the record commit + expectTaskCommitRecord(anyTimes, commitSuccess); + } return sent; } diff --git a/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/WorkerTest.java b/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/WorkerTest.java index 77238e9aaad11..06cdae85a637a 100644 --- a/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/WorkerTest.java +++ b/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/WorkerTest.java @@ -69,6 +69,7 @@ import java.util.HashSet; import java.util.List; import java.util.Map; +import java.util.concurrent.ExecutorService; import static org.apache.kafka.connect.runtime.errors.RetryWithToleranceOperatorTest.NOOP_OPERATOR; import static org.easymock.EasyMock.anyObject; @@ -112,6 +113,7 @@ public class WorkerTest extends ThreadedTest { @Mock private Converter taskKeyConverter; @Mock private Converter taskValueConverter; @Mock private HeaderConverter taskHeaderConverter; + @Mock private ExecutorService executorService; @Before public void setup() { @@ -520,8 +522,7 @@ public void testAddRemoveTask() throws Exception { expectTaskValueConverters(ClassLoaderUsage.CURRENT_CLASSLOADER, taskValueConverter); expectTaskHeaderConverter(ClassLoaderUsage.CURRENT_CLASSLOADER, taskHeaderConverter); - workerTask.run(); - EasyMock.expectLastCall(); + EasyMock.expect(executorService.submit(workerTask)).andReturn(null); EasyMock.expect(plugins.delegatingLoader()).andReturn(delegatingLoader); EasyMock.expect(delegatingLoader.connectorLoader(WorkerTestConnector.class.getName())) @@ -545,7 +546,7 @@ public void testAddRemoveTask() throws Exception { PowerMock.replayAll(); - worker = new Worker(WORKER_ID, new MockTime(), plugins, config, offsetBackingStore); + worker = new Worker(WORKER_ID, new MockTime(), plugins, config, offsetBackingStore, executorService); worker.start(); assertStatistics(worker, 0, 0); assertStartupStatistics(worker, 0, 0, 0, 0); @@ -662,8 +663,7 @@ public void testCleanupTasksOnStop() throws Exception { expectTaskHeaderConverter(ClassLoaderUsage.CURRENT_CLASSLOADER, null); expectTaskHeaderConverter(ClassLoaderUsage.PLUGINS, taskHeaderConverter); - workerTask.run(); - EasyMock.expectLastCall(); + EasyMock.expect(executorService.submit(workerTask)).andReturn(null); EasyMock.expect(plugins.delegatingLoader()).andReturn(delegatingLoader); EasyMock.expect(delegatingLoader.connectorLoader(WorkerTestConnector.class.getName())) @@ -689,7 +689,7 @@ public void testCleanupTasksOnStop() throws Exception { PowerMock.replayAll(); - worker = new Worker(WORKER_ID, new MockTime(), plugins, config, offsetBackingStore); + worker = new Worker(WORKER_ID, new MockTime(), plugins, config, offsetBackingStore, executorService); worker.start(); assertStatistics(worker, 0, 0); worker.startTask(TASK_ID, ClusterConfigState.EMPTY, anyConnectorConfigMap(), origProps, taskStatusListener, TargetState.STARTED); @@ -755,8 +755,7 @@ public void testConverterOverrides() throws Exception { expectTaskHeaderConverter(ClassLoaderUsage.CURRENT_CLASSLOADER, null); expectTaskHeaderConverter(ClassLoaderUsage.PLUGINS, taskHeaderConverter); - workerTask.run(); - EasyMock.expectLastCall(); + EasyMock.expect(executorService.submit(workerTask)).andReturn(null); EasyMock.expect(plugins.delegatingLoader()).andReturn(delegatingLoader); EasyMock.expect(delegatingLoader.connectorLoader(WorkerTestConnector.class.getName())) @@ -780,7 +779,7 @@ public void testConverterOverrides() throws Exception { PowerMock.replayAll(); - worker = new Worker(WORKER_ID, new MockTime(), plugins, config, offsetBackingStore); + worker = new Worker(WORKER_ID, new MockTime(), plugins, config, offsetBackingStore, executorService); worker.start(); assertStatistics(worker, 0, 0); assertEquals(Collections.emptySet(), worker.taskIds()); diff --git a/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/distributed/DistributedHerderTest.java b/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/distributed/DistributedHerderTest.java index 911afe7ec2f41..25c1da8c5127a 100644 --- a/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/distributed/DistributedHerderTest.java +++ b/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/distributed/DistributedHerderTest.java @@ -34,6 +34,7 @@ import org.apache.kafka.connect.runtime.TaskConfig; import org.apache.kafka.connect.runtime.Worker; import org.apache.kafka.connect.runtime.WorkerConfig; +import org.apache.kafka.connect.runtime.WorkerConfigTransformer; import org.apache.kafka.connect.runtime.distributed.DistributedHerder.HerderMetrics; import org.apache.kafka.connect.runtime.isolation.DelegatingClassLoader; import org.apache.kafka.connect.runtime.isolation.PluginClassLoader; @@ -158,6 +159,7 @@ public class DistributedHerderTest { private DistributedHerder herder; private MockConnectMetrics metrics; @Mock private Worker worker; + @Mock private WorkerConfigTransformer transformer; @Mock private Callback> putConnectorCallback; @Mock private Plugins plugins; @@ -356,6 +358,9 @@ public void testCreateConnector() throws Exception { // config validation Connector connectorMock = PowerMock.createMock(SourceConnector.class); + EasyMock.expect(worker.configTransformer()).andReturn(transformer).times(2); + final Capture> configCapture = EasyMock.newCapture(); + EasyMock.expect(transformer.transform(EasyMock.capture(configCapture))).andAnswer(configCapture::getValue); EasyMock.expect(worker.getPlugins()).andReturn(plugins).times(3); EasyMock.expect(plugins.compareAndSwapLoaders(connectorMock)).andReturn(delegatingLoader); EasyMock.expect(plugins.newConnector(EasyMock.anyString())).andReturn(connectorMock); @@ -399,6 +404,9 @@ public void testCreateConnectorFailedBasicValidation() throws Exception { // config validation Connector connectorMock = PowerMock.createMock(SourceConnector.class); + EasyMock.expect(worker.configTransformer()).andReturn(transformer).times(2); + final Capture> configCapture = EasyMock.newCapture(); + EasyMock.expect(transformer.transform(EasyMock.capture(configCapture))).andAnswer(configCapture::getValue); EasyMock.expect(worker.getPlugins()).andReturn(plugins).times(3); EasyMock.expect(plugins.compareAndSwapLoaders(connectorMock)).andReturn(delegatingLoader); EasyMock.expect(plugins.newConnector(EasyMock.anyString())).andReturn(connectorMock); @@ -444,6 +452,9 @@ public void testCreateConnectorFailedCustomValidation() throws Exception { // config validation Connector connectorMock = PowerMock.createMock(SourceConnector.class); + EasyMock.expect(worker.configTransformer()).andReturn(transformer).times(2); + final Capture> configCapture = EasyMock.newCapture(); + EasyMock.expect(transformer.transform(EasyMock.capture(configCapture))).andAnswer(configCapture::getValue); EasyMock.expect(worker.getPlugins()).andReturn(plugins).times(3); EasyMock.expect(plugins.compareAndSwapLoaders(connectorMock)).andReturn(delegatingLoader); EasyMock.expect(plugins.newConnector(EasyMock.anyString())).andReturn(connectorMock); @@ -495,6 +506,9 @@ public void testConnectorNameConflictsWithWorkerGroupId() throws Exception { // config validation Connector connectorMock = PowerMock.createMock(SinkConnector.class); + EasyMock.expect(worker.configTransformer()).andReturn(transformer).times(2); + final Capture> configCapture = EasyMock.newCapture(); + EasyMock.expect(transformer.transform(EasyMock.capture(configCapture))).andAnswer(configCapture::getValue); EasyMock.expect(worker.getPlugins()).andReturn(plugins).times(3); EasyMock.expect(plugins.compareAndSwapLoaders(connectorMock)).andReturn(delegatingLoader); EasyMock.expect(plugins.newConnector(EasyMock.anyString())).andReturn(connectorMock); @@ -530,6 +544,9 @@ public void testConnectorNameConflictsWithWorkerGroupId() throws Exception { @Test public void testCreateConnectorAlreadyExists() throws Exception { EasyMock.expect(member.memberId()).andStubReturn("leader"); + EasyMock.expect(worker.configTransformer()).andReturn(transformer).times(2); + final Capture> configCapture = EasyMock.newCapture(); + EasyMock.expect(transformer.transform(EasyMock.capture(configCapture))).andAnswer(configCapture::getValue); EasyMock.expect(worker.getPlugins()).andReturn(plugins); EasyMock.expect(plugins.newConnector(EasyMock.anyString())).andReturn(null); expectRebalance(1, Collections.emptyList(), Collections.emptyList()); @@ -1276,7 +1293,16 @@ public void testAccessors() throws Exception { EasyMock.expect(member.memberId()).andStubReturn("leader"); EasyMock.expect(worker.getPlugins()).andReturn(plugins).anyTimes(); expectRebalance(1, Collections.emptyList(), Collections.emptyList()); - expectPostRebalanceCatchup(SNAPSHOT); + + WorkerConfigTransformer configTransformer = EasyMock.mock(WorkerConfigTransformer.class); + EasyMock.expect(configTransformer.transform(EasyMock.eq(CONN1), EasyMock.anyObject())) + .andThrow(new AssertionError("Config transformation should not occur when requesting connector or task info")); + EasyMock.replay(configTransformer); + ClusterConfigState snapshotWithTransform = new ClusterConfigState(1, Collections.singletonMap(CONN1, 3), + Collections.singletonMap(CONN1, CONN1_CONFIG), Collections.singletonMap(CONN1, TargetState.STARTED), + TASK_CONFIGS_MAP, Collections.emptySet(), configTransformer); + + expectPostRebalanceCatchup(snapshotWithTransform); member.wakeup(); @@ -1339,6 +1365,9 @@ public void testPutConnectorConfig() throws Exception { // config validation Connector connectorMock = PowerMock.createMock(SourceConnector.class); + EasyMock.expect(worker.configTransformer()).andReturn(transformer).times(2); + final Capture> configCapture = EasyMock.newCapture(); + EasyMock.expect(transformer.transform(EasyMock.capture(configCapture))).andAnswer(configCapture::getValue); EasyMock.expect(worker.getPlugins()).andReturn(plugins).anyTimes(); EasyMock.expect(plugins.compareAndSwapLoaders(connectorMock)).andReturn(delegatingLoader); EasyMock.expect(plugins.newConnector(EasyMock.anyString())).andReturn(connectorMock); diff --git a/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/errors/ErrorReporterTest.java b/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/errors/ErrorReporterTest.java index b5410d071d8a9..00a922f76ad97 100644 --- a/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/errors/ErrorReporterTest.java +++ b/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/errors/ErrorReporterTest.java @@ -18,7 +18,10 @@ import org.apache.kafka.clients.consumer.ConsumerRecord; 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.header.Header; +import org.apache.kafka.connect.errors.ConnectException; import org.apache.kafka.connect.json.JsonConverter; import org.apache.kafka.connect.runtime.ConnectMetrics; import org.apache.kafka.connect.runtime.ConnectorConfig; @@ -26,6 +29,7 @@ import org.apache.kafka.connect.runtime.SinkConnectorConfig; import org.apache.kafka.connect.runtime.isolation.Plugins; import org.apache.kafka.connect.sink.SinkTask; +import org.apache.kafka.connect.transforms.Transformation; import org.apache.kafka.connect.util.ConnectorTaskId; import org.easymock.EasyMock; import org.easymock.Mock; @@ -43,8 +47,20 @@ import static java.util.Collections.emptyMap; import static java.util.Collections.singletonMap; +import static org.apache.kafka.connect.runtime.errors.DeadLetterQueueReporter.ERROR_HEADER_CONNECTOR_NAME; +import static org.apache.kafka.connect.runtime.errors.DeadLetterQueueReporter.ERROR_HEADER_EXCEPTION; +import static org.apache.kafka.connect.runtime.errors.DeadLetterQueueReporter.ERROR_HEADER_EXCEPTION_MESSAGE; +import static org.apache.kafka.connect.runtime.errors.DeadLetterQueueReporter.ERROR_HEADER_EXCEPTION_STACK_TRACE; +import static org.apache.kafka.connect.runtime.errors.DeadLetterQueueReporter.ERROR_HEADER_EXECUTING_CLASS; +import static org.apache.kafka.connect.runtime.errors.DeadLetterQueueReporter.ERROR_HEADER_ORIG_OFFSET; +import static org.apache.kafka.connect.runtime.errors.DeadLetterQueueReporter.ERROR_HEADER_ORIG_PARTITION; +import static org.apache.kafka.connect.runtime.errors.DeadLetterQueueReporter.ERROR_HEADER_ORIG_TOPIC; +import static org.apache.kafka.connect.runtime.errors.DeadLetterQueueReporter.ERROR_HEADER_STAGE; +import static org.apache.kafka.connect.runtime.errors.DeadLetterQueueReporter.ERROR_HEADER_TASK_ID; import static org.easymock.EasyMock.replay; import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertNull; +import static org.junit.Assert.assertTrue; @RunWith(PowerMockRunner.class) @PowerMockIgnore("javax.management.*") @@ -79,10 +95,15 @@ public void tearDown() { } } + @Test(expected = NullPointerException.class) + public void initializeDLQWithNullMetrics() { + new DeadLetterQueueReporter(producer, config(emptyMap()), TASK_ID, null); + } + @Test public void testDLQConfigWithEmptyTopicName() { - DeadLetterQueueReporter deadLetterQueueReporter = new DeadLetterQueueReporter(producer, config(emptyMap())); - deadLetterQueueReporter.metrics(errorHandlingMetrics); + DeadLetterQueueReporter deadLetterQueueReporter = new DeadLetterQueueReporter( + producer, config(emptyMap()), TASK_ID, errorHandlingMetrics); ProcessingContext context = processingContext(); @@ -96,8 +117,8 @@ public void testDLQConfigWithEmptyTopicName() { @Test public void testDLQConfigWithValidTopicName() { - DeadLetterQueueReporter deadLetterQueueReporter = new DeadLetterQueueReporter(producer, config(singletonMap(SinkConnectorConfig.DLQ_TOPIC_NAME_CONFIG, DLQ_TOPIC))); - deadLetterQueueReporter.metrics(errorHandlingMetrics); + DeadLetterQueueReporter deadLetterQueueReporter = new DeadLetterQueueReporter( + producer, config(singletonMap(SinkConnectorConfig.DLQ_TOPIC_NAME_CONFIG, DLQ_TOPIC)), TASK_ID, errorHandlingMetrics); ProcessingContext context = processingContext(); @@ -111,8 +132,8 @@ public void testDLQConfigWithValidTopicName() { @Test public void testReportDLQTwice() { - DeadLetterQueueReporter deadLetterQueueReporter = new DeadLetterQueueReporter(producer, config(singletonMap(SinkConnectorConfig.DLQ_TOPIC_NAME_CONFIG, DLQ_TOPIC))); - deadLetterQueueReporter.metrics(errorHandlingMetrics); + DeadLetterQueueReporter deadLetterQueueReporter = new DeadLetterQueueReporter( + producer, config(singletonMap(SinkConnectorConfig.DLQ_TOPIC_NAME_CONFIG, DLQ_TOPIC)), TASK_ID, errorHandlingMetrics); ProcessingContext context = processingContext(); @@ -127,8 +148,7 @@ public void testReportDLQTwice() { @Test public void testLogOnDisabledLogReporter() { - LogReporter logReporter = new LogReporter(TASK_ID, config(emptyMap())); - logReporter.metrics(errorHandlingMetrics); + LogReporter logReporter = new LogReporter(TASK_ID, config(emptyMap()), errorHandlingMetrics); ProcessingContext context = processingContext(); context.error(new RuntimeException()); @@ -140,8 +160,7 @@ public void testLogOnDisabledLogReporter() { @Test public void testLogOnEnabledLogReporter() { - LogReporter logReporter = new LogReporter(TASK_ID, config(singletonMap(ConnectorConfig.ERRORS_LOG_ENABLE_CONFIG, "true"))); - logReporter.metrics(errorHandlingMetrics); + LogReporter logReporter = new LogReporter(TASK_ID, config(singletonMap(ConnectorConfig.ERRORS_LOG_ENABLE_CONFIG, "true")), errorHandlingMetrics); ProcessingContext context = processingContext(); context.error(new RuntimeException()); @@ -153,8 +172,7 @@ public void testLogOnEnabledLogReporter() { @Test public void testLogMessageWithNoRecords() { - LogReporter logReporter = new LogReporter(TASK_ID, config(singletonMap(ConnectorConfig.ERRORS_LOG_ENABLE_CONFIG, "true"))); - logReporter.metrics(errorHandlingMetrics); + LogReporter logReporter = new LogReporter(TASK_ID, config(singletonMap(ConnectorConfig.ERRORS_LOG_ENABLE_CONFIG, "true")), errorHandlingMetrics); ProcessingContext context = processingContext(); @@ -169,8 +187,7 @@ public void testLogMessageWithSinkRecords() { props.put(ConnectorConfig.ERRORS_LOG_ENABLE_CONFIG, "true"); props.put(ConnectorConfig.ERRORS_LOG_INCLUDE_MESSAGES_CONFIG, "true"); - LogReporter logReporter = new LogReporter(TASK_ID, config(props)); - logReporter.metrics(errorHandlingMetrics); + LogReporter logReporter = new LogReporter(TASK_ID, config(props), errorHandlingMetrics); ProcessingContext context = processingContext(); @@ -180,6 +197,102 @@ public void testLogMessageWithSinkRecords() { "partition=5, offset=100}.", msg); } + @Test + public void testSetDLQConfigs() { + SinkConnectorConfig configuration = config(singletonMap(SinkConnectorConfig.DLQ_TOPIC_NAME_CONFIG, DLQ_TOPIC)); + assertEquals(configuration.dlqTopicName(), DLQ_TOPIC); + + configuration = config(singletonMap(SinkConnectorConfig.DLQ_TOPIC_REPLICATION_FACTOR_CONFIG, "7")); + assertEquals(configuration.dlqTopicReplicationFactor(), 7); + } + + @Test + public void testDlqHeaderConsumerRecord() { + Map props = new HashMap<>(); + props.put(SinkConnectorConfig.DLQ_TOPIC_NAME_CONFIG, DLQ_TOPIC); + props.put(SinkConnectorConfig.DLQ_CONTEXT_HEADERS_ENABLE_CONFIG, "true"); + DeadLetterQueueReporter deadLetterQueueReporter = new DeadLetterQueueReporter(producer, config(props), TASK_ID, errorHandlingMetrics); + + ProcessingContext context = new ProcessingContext(); + context.consumerRecord(new ConsumerRecord<>("source-topic", 7, 10, "source-key".getBytes(), "source-value".getBytes())); + context.currentContext(Stage.TRANSFORMATION, Transformation.class); + context.error(new ConnectException("Test Exception")); + + ProducerRecord producerRecord = new ProducerRecord<>(DLQ_TOPIC, "source-key".getBytes(), "source-value".getBytes()); + + deadLetterQueueReporter.populateContextHeaders(producerRecord, context); + assertEquals("source-topic", headerValue(producerRecord, ERROR_HEADER_ORIG_TOPIC)); + assertEquals("7", headerValue(producerRecord, ERROR_HEADER_ORIG_PARTITION)); + assertEquals("10", headerValue(producerRecord, ERROR_HEADER_ORIG_OFFSET)); + assertEquals(TASK_ID.connector(), headerValue(producerRecord, ERROR_HEADER_CONNECTOR_NAME)); + assertEquals(String.valueOf(TASK_ID.task()), headerValue(producerRecord, ERROR_HEADER_TASK_ID)); + assertEquals(Stage.TRANSFORMATION.name(), headerValue(producerRecord, ERROR_HEADER_STAGE)); + assertEquals(Transformation.class.getName(), headerValue(producerRecord, ERROR_HEADER_EXECUTING_CLASS)); + assertEquals(ConnectException.class.getName(), headerValue(producerRecord, ERROR_HEADER_EXCEPTION)); + assertEquals("Test Exception", headerValue(producerRecord, ERROR_HEADER_EXCEPTION_MESSAGE)); + assertTrue(headerValue(producerRecord, ERROR_HEADER_EXCEPTION_STACK_TRACE).length() > 0); + assertTrue(headerValue(producerRecord, ERROR_HEADER_EXCEPTION_STACK_TRACE).startsWith("org.apache.kafka.connect.errors.ConnectException: Test Exception")); + } + + @Test + public void testDlqHeaderOnNullExceptionMessage() { + Map props = new HashMap<>(); + props.put(SinkConnectorConfig.DLQ_TOPIC_NAME_CONFIG, DLQ_TOPIC); + props.put(SinkConnectorConfig.DLQ_CONTEXT_HEADERS_ENABLE_CONFIG, "true"); + DeadLetterQueueReporter deadLetterQueueReporter = new DeadLetterQueueReporter(producer, config(props), TASK_ID, errorHandlingMetrics); + + ProcessingContext context = new ProcessingContext(); + context.consumerRecord(new ConsumerRecord<>("source-topic", 7, 10, "source-key".getBytes(), "source-value".getBytes())); + context.currentContext(Stage.TRANSFORMATION, Transformation.class); + context.error(new NullPointerException()); + + ProducerRecord producerRecord = new ProducerRecord<>(DLQ_TOPIC, "source-key".getBytes(), "source-value".getBytes()); + + deadLetterQueueReporter.populateContextHeaders(producerRecord, context); + assertEquals("source-topic", headerValue(producerRecord, ERROR_HEADER_ORIG_TOPIC)); + assertEquals("7", headerValue(producerRecord, ERROR_HEADER_ORIG_PARTITION)); + assertEquals("10", headerValue(producerRecord, ERROR_HEADER_ORIG_OFFSET)); + assertEquals(TASK_ID.connector(), headerValue(producerRecord, ERROR_HEADER_CONNECTOR_NAME)); + assertEquals(String.valueOf(TASK_ID.task()), headerValue(producerRecord, ERROR_HEADER_TASK_ID)); + assertEquals(Stage.TRANSFORMATION.name(), headerValue(producerRecord, ERROR_HEADER_STAGE)); + assertEquals(Transformation.class.getName(), headerValue(producerRecord, ERROR_HEADER_EXECUTING_CLASS)); + assertEquals(NullPointerException.class.getName(), headerValue(producerRecord, ERROR_HEADER_EXCEPTION)); + assertNull(producerRecord.headers().lastHeader(ERROR_HEADER_EXCEPTION_MESSAGE).value()); + assertTrue(headerValue(producerRecord, ERROR_HEADER_EXCEPTION_STACK_TRACE).length() > 0); + assertTrue(headerValue(producerRecord, ERROR_HEADER_EXCEPTION_STACK_TRACE).startsWith("java.lang.NullPointerException")); + } + + @Test + public void testDlqHeaderIsAppended() { + Map props = new HashMap<>(); + props.put(SinkConnectorConfig.DLQ_TOPIC_NAME_CONFIG, DLQ_TOPIC); + props.put(SinkConnectorConfig.DLQ_CONTEXT_HEADERS_ENABLE_CONFIG, "true"); + DeadLetterQueueReporter deadLetterQueueReporter = new DeadLetterQueueReporter(producer, config(props), TASK_ID, errorHandlingMetrics); + + ProcessingContext context = new ProcessingContext(); + context.consumerRecord(new ConsumerRecord<>("source-topic", 7, 10, "source-key".getBytes(), "source-value".getBytes())); + context.currentContext(Stage.TRANSFORMATION, Transformation.class); + context.error(new ConnectException("Test Exception")); + + ProducerRecord producerRecord = new ProducerRecord<>(DLQ_TOPIC, "source-key".getBytes(), "source-value".getBytes()); + producerRecord.headers().add(ERROR_HEADER_ORIG_TOPIC, "dummy".getBytes()); + + deadLetterQueueReporter.populateContextHeaders(producerRecord, context); + int appearances = 0; + for (Header header: producerRecord.headers()) { + if (ERROR_HEADER_ORIG_TOPIC.equalsIgnoreCase(header.key())) { + appearances++; + } + } + + assertEquals("source-topic", headerValue(producerRecord, ERROR_HEADER_ORIG_TOPIC)); + assertEquals(2, appearances); + } + + private String headerValue(ProducerRecord producerRecord, String headerSuffix) { + return new String(producerRecord.headers().lastHeader(headerSuffix).value()); + } + private ProcessingContext processingContext() { ProcessingContext context = new ProcessingContext(); context.consumerRecord(new ConsumerRecord<>(TOPIC, 5, 100, new byte[]{'a', 'b'}, new byte[]{'x'})); diff --git a/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/health/ConnectClusterStateImplTest.java b/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/health/ConnectClusterStateImplTest.java new file mode 100644 index 0000000000000..78780f3db58ee --- /dev/null +++ b/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/health/ConnectClusterStateImplTest.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.connect.runtime.health; + +import org.apache.kafka.connect.errors.ConnectException; +import org.apache.kafka.connect.runtime.Herder; +import org.apache.kafka.connect.util.Callback; +import org.easymock.Capture; +import org.easymock.EasyMock; +import org.easymock.IAnswer; +import org.junit.Before; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.powermock.api.easymock.annotation.Mock; +import org.powermock.modules.junit4.PowerMockRunner; + +import java.util.Arrays; +import java.util.Collection; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.TimeoutException; + +import static org.junit.Assert.assertEquals; + +@RunWith(PowerMockRunner.class) +public class ConnectClusterStateImplTest { + + @Mock + protected Herder herder; + protected ConnectClusterStateImpl connectClusterState; + protected Collection expectedConnectors; + protected long herderRequestTimeoutMs = TimeUnit.SECONDS.toMillis(10); + + @Before + public void setUp() { + connectClusterState = new ConnectClusterStateImpl(herderRequestTimeoutMs, herder); + expectedConnectors = Arrays.asList("sink1", "source1", "source2"); + } + + @Test + public void connectors() { + Capture>> callback = EasyMock.newCapture(); + herder.connectors(EasyMock.capture(callback)); + EasyMock.expectLastCall().andAnswer(new IAnswer() { + @Override + public Void answer() { + callback.getValue().onCompletion(null, expectedConnectors); + return null; + } + }); + EasyMock.replay(herder); + assertEquals(expectedConnectors, connectClusterState.connectors()); + } + + @Test(expected = ConnectException.class) + public void connectorsFailure() { + Capture>> callback = EasyMock.newCapture(); + herder.connectors(EasyMock.capture(callback)); + EasyMock.expectLastCall().andAnswer(new IAnswer() { + @Override + public Void answer() { + Throwable timeout = new TimeoutException(); + callback.getValue().onCompletion(timeout, null); + return null; + } + }); + EasyMock.replay(herder); + connectClusterState.connectors(); + } +} diff --git a/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/isolation/DelegatingClassLoaderTest.java b/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/isolation/DelegatingClassLoaderTest.java new file mode 100644 index 0000000000000..3e346bb824623 --- /dev/null +++ b/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/isolation/DelegatingClassLoaderTest.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.connect.runtime.isolation; + +import java.util.Collections; +import org.junit.Test; + +import static org.junit.Assert.assertNotNull; +import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertTrue; + +public class DelegatingClassLoaderTest { + + @Test + public void testWhiteListedManifestResources() { + assertTrue( + DelegatingClassLoader.serviceLoaderManifestForPlugin("META-INF/services/org.apache.kafka.connect.rest.ConnectRestExtension")); + assertTrue( + DelegatingClassLoader.serviceLoaderManifestForPlugin("META-INF/services/org.apache.kafka.common.config.provider.ConfigProvider")); + } + + @Test + public void testOtherResources() { + assertFalse( + DelegatingClassLoader.serviceLoaderManifestForPlugin("META-INF/services/org.apache.kafka.connect.transforms.Transformation")); + assertFalse(DelegatingClassLoader.serviceLoaderManifestForPlugin("resource/version.properties")); + } + + @Test(expected = ClassNotFoundException.class) + public void testLoadingUnloadedPluginClass() throws ClassNotFoundException { + TestPlugins.assertAvailable(); + DelegatingClassLoader classLoader = new DelegatingClassLoader(Collections.emptyList()); + classLoader.initLoaders(); + for (String pluginClassName : TestPlugins.pluginClasses()) { + classLoader.loadClass(pluginClassName); + } + } + + @Test + public void testLoadingPluginClass() throws ClassNotFoundException { + TestPlugins.assertAvailable(); + DelegatingClassLoader classLoader = new DelegatingClassLoader(TestPlugins.pluginPath()); + classLoader.initLoaders(); + for (String pluginClassName : TestPlugins.pluginClasses()) { + assertNotNull(classLoader.loadClass(pluginClassName)); + assertNotNull(classLoader.pluginClassLoader(pluginClassName)); + } + } +} diff --git a/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/isolation/PluginUtilsTest.java b/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/isolation/PluginUtilsTest.java index 4bc6e15898f3c..191eec4449d6b 100644 --- a/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/isolation/PluginUtilsTest.java +++ b/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/isolation/PluginUtilsTest.java @@ -114,6 +114,9 @@ public void testConnectFrameworkClasses() throws Exception { assertFalse(PluginUtils.shouldLoadInIsolation( "org.apache.kafka.clients.admin.KafkaAdminClient") ); + assertFalse(PluginUtils.shouldLoadInIsolation( + "org.apache.kafka.connect.rest.ConnectRestExtension") + ); } @Test @@ -143,9 +146,43 @@ public void testAllowedConnectFrameworkClasses() throws Exception { assertTrue(PluginUtils.shouldLoadInIsolation( "org.apache.kafka.connect.converters.ByteArrayConverter") ); + assertTrue(PluginUtils.shouldLoadInIsolation( + "org.apache.kafka.connect.converters.DoubleConverter") + ); + assertTrue(PluginUtils.shouldLoadInIsolation( + "org.apache.kafka.connect.converters.FloatConverter") + ); + assertTrue(PluginUtils.shouldLoadInIsolation( + "org.apache.kafka.connect.converters.IntegerConverter") + ); + assertTrue(PluginUtils.shouldLoadInIsolation( + "org.apache.kafka.connect.converters.LongConverter") + ); + assertTrue(PluginUtils.shouldLoadInIsolation( + "org.apache.kafka.connect.converters.ShortConverter") + ); assertTrue(PluginUtils.shouldLoadInIsolation( "org.apache.kafka.connect.storage.StringConverter") ); + assertTrue(PluginUtils.shouldLoadInIsolation( + "org.apache.kafka.connect.storage.SimpleHeaderConverter") + ); + assertTrue(PluginUtils.shouldLoadInIsolation( + "org.apache.kafka.connect.rest.basic.auth.extension.BasicAuthSecurityRestExtension" + )); + } + + @Test + public void testClientConfigProvider() throws Exception { + assertFalse(PluginUtils.shouldLoadInIsolation( + "org.apache.kafka.common.config.provider.ConfigProvider") + ); + assertTrue(PluginUtils.shouldLoadInIsolation( + "org.apache.kafka.common.config.provider.FileConfigProvider") + ); + assertTrue(PluginUtils.shouldLoadInIsolation( + "org.apache.kafka.common.config.provider.FutureConfigProvider") + ); } @Test diff --git a/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/isolation/PluginsTest.java b/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/isolation/PluginsTest.java index 5c8aa29f338c3..6afc93204c247 100644 --- a/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/isolation/PluginsTest.java +++ b/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/isolation/PluginsTest.java @@ -17,11 +17,16 @@ package org.apache.kafka.connect.runtime.isolation; +import java.util.Collections; +import java.util.Map.Entry; import org.apache.kafka.common.Configurable; import org.apache.kafka.common.config.AbstractConfig; import org.apache.kafka.common.config.ConfigDef; +import org.apache.kafka.common.config.ConfigException; +import org.apache.kafka.common.config.provider.ConfigProvider; import org.apache.kafka.connect.data.Schema; import org.apache.kafka.connect.data.SchemaAndValue; +import org.apache.kafka.connect.errors.ConnectException; import org.apache.kafka.connect.json.JsonConverter; import org.apache.kafka.connect.json.JsonConverterConfig; import org.apache.kafka.connect.rest.ConnectRestExtension; @@ -34,7 +39,6 @@ import org.apache.kafka.connect.storage.HeaderConverter; import org.apache.kafka.connect.storage.SimpleHeaderConverter; import org.junit.Before; -import org.junit.BeforeClass; import org.junit.Test; import java.io.IOException; @@ -45,30 +49,26 @@ import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertNotNull; import static org.junit.Assert.assertNull; +import static org.junit.Assert.assertSame; import static org.junit.Assert.assertTrue; public class PluginsTest { - private static Map pluginProps; - private static Plugins plugins; + private Plugins plugins; private Map props; private AbstractConfig config; private TestConverter converter; private TestHeaderConverter headerConverter; private TestInternalConverter internalConverter; - @BeforeClass - public static void beforeAll() { - pluginProps = new HashMap<>(); - - // Set up the plugins to have no additional plugin directories. - // This won't allow us to test classpath isolation, but it will allow us to test some of the utility methods. - pluginProps.put(WorkerConfig.PLUGIN_PATH_CONFIG, ""); - plugins = new Plugins(pluginProps); - } - + @SuppressWarnings("deprecation") @Before public void setup() { + Map pluginProps = new HashMap<>(); + + // Set up the plugins with some test plugins to test isolation + pluginProps.put(WorkerConfig.PLUGIN_PATH_CONFIG, String.join(",", TestPlugins.pluginPath())); + plugins = new Plugins(pluginProps); props = new HashMap<>(pluginProps); props.put(WorkerConfig.KEY_CONVERTER_CLASS_CONFIG, TestConverter.class.getName()); props.put(WorkerConfig.VALUE_CONVERTER_CLASS_CONFIG, TestConverter.class.getName()); @@ -183,6 +183,186 @@ public void shouldInstantiateAndConfigureDefaultHeaderConverter() { assertTrue(headerConverter instanceof SimpleHeaderConverter); } + @Test(expected = ConnectException.class) + public void shouldThrowIfPluginThrows() { + TestPlugins.assertAvailable(); + + plugins.newPlugin( + TestPlugins.ALWAYS_THROW_EXCEPTION, + new AbstractConfig(new ConfigDef(), Collections.emptyMap()), + Converter.class + ); + } + + @Test + public void shouldShareStaticValuesBetweenSamePlugin() { + // Plugins are not isolated from other instances of their own class. + TestPlugins.assertAvailable(); + Converter firstPlugin = plugins.newPlugin( + TestPlugins.ALIASED_STATIC_FIELD, + new AbstractConfig(new ConfigDef(), Collections.emptyMap()), + Converter.class + ); + + assertInstanceOf(SamplingTestPlugin.class, firstPlugin, "Cannot collect samples"); + + Converter secondPlugin = plugins.newPlugin( + TestPlugins.ALIASED_STATIC_FIELD, + new AbstractConfig(new ConfigDef(), Collections.emptyMap()), + Converter.class + ); + + assertInstanceOf(SamplingTestPlugin.class, secondPlugin, "Cannot collect samples"); + assertSame( + ((SamplingTestPlugin) firstPlugin).otherSamples(), + ((SamplingTestPlugin) secondPlugin).otherSamples() + ); + } + + @Test + public void newPluginShouldServiceLoadWithPluginClassLoader() { + TestPlugins.assertAvailable(); + Converter plugin = plugins.newPlugin( + TestPlugins.SERVICE_LOADER, + new AbstractConfig(new ConfigDef(), Collections.emptyMap()), + Converter.class + ); + + assertInstanceOf(SamplingTestPlugin.class, plugin, "Cannot collect samples"); + Map samples = ((SamplingTestPlugin) plugin).flatten(); + // Assert that the service loaded subclass is found in both environments + assertTrue(samples.containsKey("ServiceLoadedSubclass.static")); + assertTrue(samples.containsKey("ServiceLoadedSubclass.dynamic")); + assertPluginClassLoaderAlwaysActive(samples); + } + + @Test + public void newPluginShouldInstantiateWithPluginClassLoader() { + TestPlugins.assertAvailable(); + Converter plugin = plugins.newPlugin( + TestPlugins.ALIASED_STATIC_FIELD, + new AbstractConfig(new ConfigDef(), Collections.emptyMap()), + Converter.class + ); + + assertInstanceOf(SamplingTestPlugin.class, plugin, "Cannot collect samples"); + Map samples = ((SamplingTestPlugin) plugin).flatten(); + assertPluginClassLoaderAlwaysActive(samples); + } + + @Test(expected = ConfigException.class) + public void shouldFailToFindConverterInCurrentClassloader() { + TestPlugins.assertAvailable(); + props.put(WorkerConfig.KEY_CONVERTER_CLASS_CONFIG, TestPlugins.SAMPLING_CONVERTER); + createConfig(); + } + + @Test + public void newConverterShouldConfigureWithPluginClassLoader() { + TestPlugins.assertAvailable(); + props.put(WorkerConfig.KEY_CONVERTER_CLASS_CONFIG, TestPlugins.SAMPLING_CONVERTER); + ClassLoader classLoader = plugins.delegatingLoader().pluginClassLoader(TestPlugins.SAMPLING_CONVERTER); + ClassLoader savedLoader = Plugins.compareAndSwapLoaders(classLoader); + createConfig(); + Plugins.compareAndSwapLoaders(savedLoader); + + Converter plugin = plugins.newConverter( + config, + WorkerConfig.KEY_CONVERTER_CLASS_CONFIG, + ClassLoaderUsage.PLUGINS + ); + + assertInstanceOf(SamplingTestPlugin.class, plugin, "Cannot collect samples"); + Map samples = ((SamplingTestPlugin) plugin).flatten(); + assertTrue(samples.containsKey("configure")); + assertPluginClassLoaderAlwaysActive(samples); + } + + @Test + public void newConfigProviderShouldConfigureWithPluginClassLoader() { + TestPlugins.assertAvailable(); + String providerPrefix = "some.provider"; + props.put(providerPrefix + ".class", TestPlugins.SAMPLING_CONFIG_PROVIDER); + + PluginClassLoader classLoader = plugins.delegatingLoader().pluginClassLoader(TestPlugins.SAMPLING_CONFIG_PROVIDER); + assertNotNull(classLoader); + ClassLoader savedLoader = Plugins.compareAndSwapLoaders(classLoader); + createConfig(); + Plugins.compareAndSwapLoaders(savedLoader); + + ConfigProvider plugin = plugins.newConfigProvider( + config, + providerPrefix, + ClassLoaderUsage.PLUGINS + ); + + assertInstanceOf(SamplingTestPlugin.class, plugin, "Cannot collect samples"); + Map samples = ((SamplingTestPlugin) plugin).flatten(); + assertTrue(samples.containsKey("configure")); + assertPluginClassLoaderAlwaysActive(samples); + } + + @Test + public void newHeaderConverterShouldConfigureWithPluginClassLoader() { + TestPlugins.assertAvailable(); + props.put(WorkerConfig.HEADER_CONVERTER_CLASS_CONFIG, TestPlugins.SAMPLING_HEADER_CONVERTER); + ClassLoader classLoader = plugins.delegatingLoader().pluginClassLoader(TestPlugins.SAMPLING_HEADER_CONVERTER); + ClassLoader savedLoader = Plugins.compareAndSwapLoaders(classLoader); + createConfig(); + Plugins.compareAndSwapLoaders(savedLoader); + + HeaderConverter plugin = plugins.newHeaderConverter( + config, + WorkerConfig.HEADER_CONVERTER_CLASS_CONFIG, + ClassLoaderUsage.PLUGINS + ); + + assertInstanceOf(SamplingTestPlugin.class, plugin, "Cannot collect samples"); + Map samples = ((SamplingTestPlugin) plugin).flatten(); + assertTrue(samples.containsKey("configure")); // HeaderConverter::configure was called + assertPluginClassLoaderAlwaysActive(samples); + } + + @Test + public void newPluginsShouldConfigureWithPluginClassLoader() { + TestPlugins.assertAvailable(); + List configurables = plugins.newPlugins( + Collections.singletonList(TestPlugins.SAMPLING_CONFIGURABLE), + config, + Configurable.class + ); + assertEquals(1, configurables.size()); + Configurable plugin = configurables.get(0); + + assertInstanceOf(SamplingTestPlugin.class, plugin, "Cannot collect samples"); + Map samples = ((SamplingTestPlugin) plugin).flatten(); + assertTrue(samples.containsKey("configure")); // Configurable::configure was called + assertPluginClassLoaderAlwaysActive(samples); + } + + public static void assertPluginClassLoaderAlwaysActive(Map samples) { + for (Entry e : samples.entrySet()) { + String sampleName = "\"" + e.getKey() + "\" (" + e.getValue() + ")"; + assertInstanceOf( + PluginClassLoader.class, + e.getValue().staticClassloader(), + sampleName + " has incorrect static classloader" + ); + assertInstanceOf( + PluginClassLoader.class, + e.getValue().classloader(), + sampleName + " has incorrect dynamic classloader" + ); + } + } + + public static void assertInstanceOf(Class expected, Object actual, String message) { + assertTrue( + "Expected an instance of " + expected.getSimpleName() + ", found " + actual + " instead: " + message, + expected.isInstance(actual) + ); + } + protected void instantiateAndConfigureConverter(String configPropName, ClassLoaderUsage classLoaderUsage) { converter = (TestConverter) plugins.newConverter(config, configPropName, classLoaderUsage); assertNotNull(converter); diff --git a/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/isolation/SamplingTestPlugin.java b/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/isolation/SamplingTestPlugin.java new file mode 100644 index 0000000000000..bcf8881898e8e --- /dev/null +++ b/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/isolation/SamplingTestPlugin.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.connect.runtime.isolation; + +import java.util.Collections; +import java.util.HashMap; +import java.util.Map; +import java.util.Map.Entry; + +/** + * Base class for plugins so we can sample information about their initialization + */ +public abstract class SamplingTestPlugin { + + /** + * @return the ClassLoader used to statically initialize this plugin class + */ + public abstract ClassLoader staticClassloader(); + + /** + * @return the ClassLoader used to initialize this plugin instance + */ + public abstract ClassLoader classloader(); + + /** + * @return a group of other SamplingTestPlugin instances known by this plugin + * This should only return direct children, and not reference this instance directly + */ + public Map otherSamples() { + return Collections.emptyMap(); + } + + /** + * @return a flattened list of child samples including this entry keyed as "this" + */ + public Map flatten() { + Map out = new HashMap<>(); + Map otherSamples = otherSamples(); + if (otherSamples != null) { + for (Entry child : otherSamples.entrySet()) { + for (Entry flattened : child.getValue().flatten().entrySet()) { + String key = child.getKey(); + if (flattened.getKey().length() > 0) { + key += "." + flattened.getKey(); + } + out.put(key, flattened.getValue()); + } + } + } + out.put("", this); + return out; + } + + /** + * Log the parent method call as a child sample. + * Stores only the last invocation of each method if there are multiple invocations. + * @param samples The collection of samples to which this method call should be added + */ + public void logMethodCall(Map samples) { + StackTraceElement[] stackTraces = Thread.currentThread().getStackTrace(); + if (stackTraces.length < 2) { + return; + } + // 0 is inside getStackTrace + // 1 is this method + // 2 is our caller method + StackTraceElement caller = stackTraces[2]; + + samples.put(caller.getMethodName(), new MethodCallSample( + caller, + Thread.currentThread().getContextClassLoader(), + getClass().getClassLoader() + )); + } + + public static class MethodCallSample extends SamplingTestPlugin { + + private final StackTraceElement caller; + private final ClassLoader staticClassLoader; + private final ClassLoader dynamicClassLoader; + + public MethodCallSample( + StackTraceElement caller, + ClassLoader staticClassLoader, + ClassLoader dynamicClassLoader + ) { + this.caller = caller; + this.staticClassLoader = staticClassLoader; + this.dynamicClassLoader = dynamicClassLoader; + } + + @Override + public ClassLoader staticClassloader() { + return staticClassLoader; + } + + @Override + public ClassLoader classloader() { + return dynamicClassLoader; + } + + @Override + public String toString() { + return caller.toString(); + } + } +} diff --git a/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/isolation/TestPlugins.java b/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/isolation/TestPlugins.java new file mode 100644 index 0000000000000..9561ffb0f5b14 --- /dev/null +++ b/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/isolation/TestPlugins.java @@ -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 org.apache.kafka.connect.runtime.isolation; + +import java.io.BufferedInputStream; +import java.io.File; +import java.io.FileInputStream; +import java.io.FileOutputStream; +import java.io.IOException; +import java.io.InputStream; +import java.io.StringWriter; +import java.net.URL; +import java.nio.file.Files; +import java.nio.file.Path; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collections; +import java.util.Comparator; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.jar.Attributes; +import java.util.jar.JarEntry; +import java.util.jar.JarOutputStream; +import java.util.jar.Manifest; +import java.util.stream.Collectors; +import javax.tools.JavaCompiler; +import javax.tools.StandardJavaFileManager; +import javax.tools.ToolProvider; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * Utility class for constructing test plugins for Connect. + * + *

Plugins are built from their source under resources/test-plugins/ and placed into temporary + * jar files that are deleted when the process exits. + * + *

To add a plugin, create the source files in the resource tree, and edit this class to build + * that plugin during initialization. For example, the plugin class {@literal package.Class} should + * be placed in {@literal resources/test-plugins/something/package/Class.java} and loaded using + * {@code createPluginJar("something")}. The class name, contents, and plugin directory can take + * any value you need for testing. + * + *

To use this class in your tests, make sure to first call + * {@link TestPlugins#assertAvailable()} to verify that the plugins initialized correctly. + * Otherwise, exceptions during the plugin build are not propagated, and may invalidate your test. + * You can access the list of plugin jars for assembling a {@literal plugin.path}, and reference + * the names of the different plugins directly via the exposed constants. + */ +public class TestPlugins { + + /** + * Class name of a plugin which will always throw an exception during loading + */ + public static final String ALWAYS_THROW_EXCEPTION = "test.plugins.AlwaysThrowException"; + /** + * Class name of a plugin which samples information about its initialization. + */ + public static final String ALIASED_STATIC_FIELD = "test.plugins.AliasedStaticField"; + /** + * Class name of a {@link org.apache.kafka.connect.storage.Converter} + * which samples information about its method calls. + */ + public static final String SAMPLING_CONVERTER = "test.plugins.SamplingConverter"; + /** + * Class name of a {@link org.apache.kafka.common.Configurable} + * which samples information about its method calls. + */ + public static final String SAMPLING_CONFIGURABLE = "test.plugins.SamplingConfigurable"; + /** + * Class name of a {@link org.apache.kafka.connect.storage.HeaderConverter} + * which samples information about its method calls. + */ + public static final String SAMPLING_HEADER_CONVERTER = "test.plugins.SamplingHeaderConverter"; + /** + * Class name of a {@link org.apache.kafka.common.config.provider.ConfigProvider} + * which samples information about its method calls. + */ + public static final String SAMPLING_CONFIG_PROVIDER = "test.plugins.SamplingConfigProvider"; + /** + * Class name of a plugin which uses a {@link java.util.ServiceLoader} + * to load internal classes, and samples information about their initialization. + */ + public static final String SERVICE_LOADER = "test.plugins.ServiceLoaderPlugin"; + + private static final Logger log = LoggerFactory.getLogger(TestPlugins.class); + private static final Map PLUGIN_JARS; + private static final Throwable INITIALIZATION_EXCEPTION; + + static { + Throwable err = null; + HashMap pluginJars = new HashMap<>(); + try { + pluginJars.put(ALWAYS_THROW_EXCEPTION, createPluginJar("always-throw-exception")); + pluginJars.put(ALIASED_STATIC_FIELD, createPluginJar("aliased-static-field")); + pluginJars.put(SAMPLING_CONVERTER, createPluginJar("sampling-converter")); + pluginJars.put(SAMPLING_CONFIGURABLE, createPluginJar("sampling-configurable")); + pluginJars.put(SAMPLING_HEADER_CONVERTER, createPluginJar("sampling-header-converter")); + pluginJars.put(SAMPLING_CONFIG_PROVIDER, createPluginJar("sampling-config-provider")); + pluginJars.put(SERVICE_LOADER, createPluginJar("service-loader")); + } catch (Throwable e) { + log.error("Could not set up plugin test jars", e); + err = e; + } + PLUGIN_JARS = Collections.unmodifiableMap(pluginJars); + INITIALIZATION_EXCEPTION = err; + } + + /** + * Ensure that the test plugin JARs were assembled without error before continuing. + * @throws AssertionError if any plugin failed to load, or no plugins were loaded. + */ + public static void assertAvailable() throws AssertionError { + if (INITIALIZATION_EXCEPTION != null) { + throw new AssertionError("TestPlugins did not initialize completely", + INITIALIZATION_EXCEPTION); + } + if (PLUGIN_JARS.isEmpty()) { + throw new AssertionError("No test plugins loaded"); + } + } + + /** + * A list of jar files containing test plugins + * @return A list of plugin jar filenames + */ + public static List pluginPath() { + return PLUGIN_JARS.values() + .stream() + .map(File::getPath) + .collect(Collectors.toList()); + } + + /** + * Get all of the classes that were successfully built by this class + * @return A list of plugin class names + */ + public static List pluginClasses() { + return new ArrayList<>(PLUGIN_JARS.keySet()); + } + + private static File createPluginJar(String resourceDir) throws IOException { + Path inputDir = resourceDirectoryPath("test-plugins/" + resourceDir); + Path binDir = Files.createTempDirectory(resourceDir + ".bin."); + compileJavaSources(inputDir, binDir); + File jarFile = Files.createTempFile(resourceDir + ".", ".jar").toFile(); + try (JarOutputStream jar = openJarFile(jarFile)) { + writeJar(jar, inputDir); + writeJar(jar, binDir); + } + removeDirectory(binDir); + jarFile.deleteOnExit(); + return jarFile; + } + + private static Path resourceDirectoryPath(String resourceDir) throws IOException { + URL resource = Thread.currentThread() + .getContextClassLoader() + .getResource(resourceDir); + if (resource == null) { + throw new IOException("Could not find test plugin resource: " + resourceDir); + } + File file = new File(resource.getFile()); + if (!file.isDirectory()) { + throw new IOException("Resource is not a directory: " + resourceDir); + } + if (!file.canRead()) { + throw new IOException("Resource directory is not readable: " + resourceDir); + } + return file.toPath(); + } + + private static JarOutputStream openJarFile(File jarFile) throws IOException { + Manifest manifest = new Manifest(); + manifest.getMainAttributes().put(Attributes.Name.MANIFEST_VERSION, "1.0"); + return new JarOutputStream(new FileOutputStream(jarFile), manifest); + } + + private static void removeDirectory(Path binDir) throws IOException { + List classFiles = Files.walk(binDir) + .sorted(Comparator.reverseOrder()) + .map(Path::toFile) + .collect(Collectors.toList()); + for (File classFile : classFiles) { + if (!classFile.delete()) { + throw new IOException("Could not delete: " + classFile); + } + } + } + + /** + * Compile a directory of .java source files into .class files + * .class files are placed into the same directory as their sources. + * + *

Dependencies between source files in this directory are resolved against one another + * and the classes present in the test environment. + * See https://stackoverflow.com/questions/1563909/ for more information. + * Additional dependencies in your plugins should be added as test scope to :connect:runtime. + * @param sourceDir Directory containing java source files + * @throws IOException if the files cannot be compiled + */ + private static void compileJavaSources(Path sourceDir, Path binDir) throws IOException { + JavaCompiler compiler = ToolProvider.getSystemJavaCompiler(); + List sourceFiles = Files.walk(sourceDir) + .filter(Files::isRegularFile) + .filter(path -> path.toFile().getName().endsWith(".java")) + .map(Path::toFile) + .collect(Collectors.toList()); + StringWriter writer = new StringWriter(); + List options = Arrays.asList( + "-d", binDir.toString() // Write class output to a different directory. + ); + + try (StandardJavaFileManager fileManager = compiler.getStandardFileManager(null, null, null)) { + boolean success = compiler.getTask( + writer, + fileManager, + null, + options, + null, + fileManager.getJavaFileObjectsFromFiles(sourceFiles) + ).call(); + if (!success) { + throw new RuntimeException("Failed to compile test plugin:\n" + writer); + } + } + } + + private static void writeJar(JarOutputStream jar, Path inputDir) throws IOException { + List paths = Files.walk(inputDir) + .filter(Files::isRegularFile) + .filter(path -> !path.toFile().getName().endsWith(".java")) + .collect(Collectors.toList()); + for (Path path : paths) { + try (InputStream in = new BufferedInputStream(new FileInputStream(path.toFile()))) { + jar.putNextEntry(new JarEntry( + inputDir.relativize(path) + .toFile() + .getPath() + .replace(File.separator, "/") + )); + byte[] buffer = new byte[1024]; + for (int count; (count = in.read(buffer)) != -1; ) { + jar.write(buffer, 0, count); + } + jar.closeEntry(); + } + } + } + +} diff --git a/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/rest/RestServerTest.java b/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/rest/RestServerTest.java index 2f8704ae4e820..7c0a16249d8ec 100644 --- a/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/rest/RestServerTest.java +++ b/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/rest/RestServerTest.java @@ -16,41 +16,44 @@ */ package org.apache.kafka.connect.runtime.rest; +import org.apache.http.HttpHost; +import org.apache.http.HttpRequest; +import org.apache.http.client.methods.CloseableHttpResponse; +import org.apache.http.client.methods.HttpGet; +import org.apache.http.client.methods.HttpOptions; +import org.apache.http.impl.client.CloseableHttpClient; +import org.apache.http.impl.client.HttpClients; import org.apache.kafka.clients.CommonClientConfigs; import org.apache.kafka.connect.rest.ConnectRestExtension; import org.apache.kafka.connect.runtime.Herder; import org.apache.kafka.connect.runtime.WorkerConfig; import org.apache.kafka.connect.runtime.distributed.DistributedConfig; import org.apache.kafka.connect.runtime.isolation.Plugins; +import org.apache.kafka.connect.runtime.standalone.StandaloneConfig; import org.apache.kafka.connect.util.Callback; import org.easymock.Capture; import org.easymock.EasyMock; -import org.easymock.IAnswer; import org.junit.After; import org.junit.Assert; import org.junit.Test; import org.junit.runner.RunWith; import org.powermock.api.easymock.PowerMock; import org.powermock.api.easymock.annotation.MockStrict; +import org.powermock.core.classloader.annotations.PowerMockIgnore; import org.powermock.modules.junit4.PowerMockRunner; -import java.net.URI; -import java.net.URISyntaxException; +import javax.ws.rs.core.MediaType; +import java.io.ByteArrayOutputStream; +import java.io.IOException; +import java.nio.charset.StandardCharsets; import java.util.Arrays; import java.util.Collection; import java.util.Collections; import java.util.HashMap; import java.util.Map; -import javax.ws.rs.client.Client; -import javax.ws.rs.client.ClientBuilder; -import javax.ws.rs.client.Invocation; -import javax.ws.rs.client.WebTarget; -import javax.ws.rs.core.Response; - -import static org.junit.Assert.assertEquals; - @RunWith(PowerMockRunner.class) +@PowerMockIgnore({"javax.net.ssl.*", "javax.security.*"}) public class RestServerTest { @MockStrict @@ -80,12 +83,12 @@ private Map baseWorkerProps() { } @Test - public void testCORSEnabled() { + public void testCORSEnabled() throws IOException { checkCORSRequest("*", "http://bar.com", "http://bar.com", "PUT"); } @Test - public void testCORSDisabled() { + public void testCORSDisabled() throws IOException { checkCORSRequest("", "http://bar.com", null, null); } @@ -155,14 +158,46 @@ public void testAdvertisedUri() { Assert.assertEquals("http://my-hostname:8080/", server.advertisedUrl().toString()); } - public void checkCORSRequest(String corsDomain, String origin, String expectedHeader, String method) { - // To be able to set the Origin, we need to toggle this flag + @Test + public void testOptionsDoesNotIncludeWadlOutput() throws IOException { + Map configMap = new HashMap<>(baseWorkerProps()); + DistributedConfig workerConfig = new DistributedConfig(configMap); + EasyMock.expect(herder.plugins()).andStubReturn(plugins); + EasyMock.expect(plugins.newPlugins(Collections.emptyList(), + workerConfig, + ConnectRestExtension.class)) + .andStubReturn(Collections.emptyList()); + PowerMock.replayAll(); + + server = new RestServer(workerConfig); + server.initializeServer(); + server.initializeResources(herder); + + HttpOptions request = new HttpOptions("/connectors"); + request.addHeader("Content-Type", MediaType.WILDCARD); + CloseableHttpClient httpClient = HttpClients.createMinimal(); + HttpHost httpHost = new HttpHost( + server.advertisedUrl().getHost(), + server.advertisedUrl().getPort() + ); + CloseableHttpResponse response = httpClient.execute(httpHost, request); + Assert.assertEquals(MediaType.TEXT_PLAIN, response.getEntity().getContentType().getValue()); + ByteArrayOutputStream baos = new ByteArrayOutputStream(); + response.getEntity().writeTo(baos); + Assert.assertArrayEquals( + request.getAllowedMethods(response).toArray(), + new String(baos.toByteArray(), StandardCharsets.UTF_8).split(", ") + ); + PowerMock.verifyAll(); + } + + public void checkCORSRequest(String corsDomain, String origin, String expectedHeader, String method) + throws IOException { Map workerProps = baseWorkerProps(); workerProps.put(WorkerConfig.ACCESS_CONTROL_ALLOW_ORIGIN_CONFIG, corsDomain); workerProps.put(WorkerConfig.ACCESS_CONTROL_ALLOW_METHODS_CONFIG, method); WorkerConfig workerConfig = new DistributedConfig(workerProps); - System.setProperty("sun.net.http.allowRestrictedHeaders", "true"); EasyMock.expect(herder.plugins()).andStubReturn(plugins); EasyMock.expect(plugins.newPlugins(Collections.EMPTY_LIST, @@ -172,74 +207,79 @@ public void checkCORSRequest(String corsDomain, String origin, String expectedHe final Capture>> connectorsCallback = EasyMock.newCapture(); herder.connectors(EasyMock.capture(connectorsCallback)); - PowerMock.expectLastCall().andAnswer(new IAnswer() { - @Override - public Object answer() throws Throwable { - connectorsCallback.getValue().onCompletion(null, Arrays.asList("a", "b")); - return null; - } + PowerMock.expectLastCall().andAnswer(() -> { + connectorsCallback.getValue().onCompletion(null, Arrays.asList("a", "b")); + return null; }); PowerMock.replayAll(); - server = new RestServer(workerConfig); - server.start(herder); - - Response response = request("/connectors") - .header("Referer", origin + "/page") - .header("Origin", origin) - .get(); - assertEquals(200, response.getStatus()); - - assertEquals(expectedHeader, response.getHeaderString("Access-Control-Allow-Origin")); - - response = request("/connector-plugins/FileStreamSource/validate") - .header("Referer", origin + "/page") - .header("Origin", origin) - .header("Access-Control-Request-Method", method) - .options(); - assertEquals(404, response.getStatus()); - assertEquals(expectedHeader, response.getHeaderString("Access-Control-Allow-Origin")); - assertEquals(method, response.getHeaderString("Access-Control-Allow-Methods")); + server.initializeServer(); + server.initializeResources(herder); + HttpRequest request = new HttpGet("/connectors"); + request.addHeader("Referer", origin + "/page"); + request.addHeader("Origin", origin); + CloseableHttpClient httpClient = HttpClients.createMinimal(); + HttpHost httpHost = new HttpHost( + server.advertisedUrl().getHost(), + server.advertisedUrl().getPort() + ); + CloseableHttpResponse response = httpClient.execute(httpHost, request); + + Assert.assertEquals(200, response.getStatusLine().getStatusCode()); + + if (expectedHeader != null) { + Assert.assertEquals(expectedHeader, + response.getFirstHeader("Access-Control-Allow-Origin").getValue()); + } + + request = new HttpOptions("/connector-plugins/FileStreamSource/validate"); + request.addHeader("Referer", origin + "/page"); + request.addHeader("Origin", origin); + request.addHeader("Access-Control-Request-Method", method); + response = httpClient.execute(httpHost, request); + Assert.assertEquals(404, response.getStatusLine().getStatusCode()); + if (expectedHeader != null) { + Assert.assertEquals(expectedHeader, + response.getFirstHeader("Access-Control-Allow-Origin").getValue()); + } + if (method != null) { + Assert.assertEquals(method, + response.getFirstHeader("Access-Control-Allow-Methods").getValue()); + } PowerMock.verifyAll(); } - protected Invocation.Builder request(String path) { - return request(path, null, null, null); - } + @Test + public void testStandaloneConfig() throws IOException { + Map workerProps = baseWorkerProps(); + workerProps.put("offset.storage.file.filename", "/tmp"); + WorkerConfig workerConfig = new StandaloneConfig(workerProps); - protected Invocation.Builder request(String path, Map queryParams) { - return request(path, null, null, queryParams); - } - protected Invocation.Builder request(String path, String templateName, Object templateValue) { - return request(path, templateName, templateValue, null); - } + EasyMock.expect(herder.plugins()).andStubReturn(plugins); + EasyMock.expect(plugins.newPlugins(Collections.EMPTY_LIST, + workerConfig, + ConnectRestExtension.class)).andStubReturn(Collections.EMPTY_LIST); - protected Invocation.Builder request(String path, String templateName, Object templateValue, - Map queryParams) { - Client client = ClientBuilder.newClient(); - WebTarget target; - URI pathUri = null; - try { - pathUri = new URI(path); - } catch (URISyntaxException e) { - // Ignore, use restConnect and assume this is a valid path part - } - if (pathUri != null && pathUri.isAbsolute()) { - target = client.target(path); - } else { - target = client.target(server.advertisedUrl()).path(path); - } - if (templateName != null && templateValue != null) { - target = target.resolveTemplate(templateName, templateValue); - } - if (queryParams != null) { - for (Map.Entry queryParam : queryParams.entrySet()) { - target = target.queryParam(queryParam.getKey(), queryParam.getValue()); - } - } - return target.request(); + final Capture>> connectorsCallback = EasyMock.newCapture(); + herder.connectors(EasyMock.capture(connectorsCallback)); + PowerMock.expectLastCall().andAnswer(() -> { + connectorsCallback.getValue().onCompletion(null, Arrays.asList("a", "b")); + return null; + }); + + PowerMock.replayAll(); + + server = new RestServer(workerConfig); + server.initializeServer(); + server.initializeResources(herder); + HttpRequest request = new HttpGet("/connectors"); + CloseableHttpClient httpClient = HttpClients.createMinimal(); + HttpHost httpHost = new HttpHost(server.advertisedUrl().getHost(), server.advertisedUrl().getPort()); + CloseableHttpResponse response = httpClient.execute(httpHost, request); + + Assert.assertEquals(200, response.getStatusLine().getStatusCode()); } } diff --git a/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/rest/resources/ConnectorPluginsResourceTest.java b/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/rest/resources/ConnectorPluginsResourceTest.java index ad360b669907b..67cae676c4277 100644 --- a/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/rest/resources/ConnectorPluginsResourceTest.java +++ b/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/rest/resources/ConnectorPluginsResourceTest.java @@ -18,6 +18,7 @@ import com.fasterxml.jackson.core.type.TypeReference; import com.fasterxml.jackson.databind.ObjectMapper; +import javax.ws.rs.core.HttpHeaders; import org.apache.kafka.common.config.Config; import org.apache.kafka.common.config.ConfigDef; import org.apache.kafka.common.config.ConfigDef.Importance; @@ -179,7 +180,7 @@ public class ConnectorPluginsResourceTest { @Before public void setUp() throws Exception { PowerMock.mockStatic(RestClient.class, - RestClient.class.getMethod("httpRequest", String.class, String.class, Object.class, TypeReference.class, WorkerConfig.class)); + RestClient.class.getMethod("httpRequest", String.class, String.class, HttpHeaders.class, Object.class, TypeReference.class, WorkerConfig.class)); plugins = PowerMock.createMock(Plugins.class); herder = PowerMock.createMock(AbstractHerder.class); diff --git a/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/rest/resources/ConnectorsResourceTest.java b/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/rest/resources/ConnectorsResourceTest.java index f84cd258fd4ef..ba5a2c35a0e78 100644 --- a/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/rest/resources/ConnectorsResourceTest.java +++ b/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/rest/resources/ConnectorsResourceTest.java @@ -18,6 +18,7 @@ import com.fasterxml.jackson.core.type.TypeReference; +import javax.ws.rs.core.HttpHeaders; import org.apache.kafka.connect.errors.AlreadyExistsException; import org.apache.kafka.connect.errors.ConnectException; import org.apache.kafka.connect.errors.NotFoundException; @@ -75,6 +76,7 @@ public class ConnectorsResourceTest { private static final String CONNECTOR_NAME_PADDING_WHITESPACES = " " + CONNECTOR_NAME + " \n "; private static final Boolean FORWARD = true; private static final Map CONNECTOR_CONFIG_SPECIAL_CHARS = new HashMap<>(); + private static final HttpHeaders NULL_HEADERS = null; static { CONNECTOR_CONFIG_SPECIAL_CHARS.put("name", CONNECTOR_NAME_SPECIAL_CHARS); CONNECTOR_CONFIG_SPECIAL_CHARS.put("sample_config", "test_config"); @@ -125,7 +127,7 @@ public class ConnectorsResourceTest { @Before public void setUp() throws NoSuchMethodException { PowerMock.mockStatic(RestClient.class, - RestClient.class.getMethod("httpRequest", String.class, String.class, Object.class, TypeReference.class, WorkerConfig.class)); + RestClient.class.getMethod("httpRequest", String.class, String.class, HttpHeaders.class, Object.class, TypeReference.class, WorkerConfig.class)); connectorsResource = new ConnectorsResource(herder, null); } @@ -142,7 +144,7 @@ public void testListConnectors() throws Throwable { PowerMock.replayAll(); - Collection connectors = connectorsResource.listConnectors(FORWARD); + Collection connectors = connectorsResource.listConnectors(FORWARD, NULL_HEADERS); // Ordering isn't guaranteed, compare sets assertEquals(new HashSet<>(Arrays.asList(CONNECTOR_NAME, CONNECTOR2_NAME)), new HashSet<>(connectors)); @@ -156,15 +158,12 @@ public void testListConnectorsNotLeader() throws Throwable { expectAndCallbackNotLeaderException(cb); // Should forward request EasyMock.expect(RestClient.httpRequest(EasyMock.eq("http://leader:8083/connectors?forward=false"), EasyMock.eq("GET"), - EasyMock.isNull(), EasyMock.anyObject(TypeReference.class), EasyMock.anyObject(WorkerConfig.class))) + EasyMock.isNull(), EasyMock.isNull(), EasyMock.anyObject(TypeReference.class), EasyMock.anyObject(WorkerConfig.class))) .andReturn(new RestClient.HttpResponse<>(200, new HashMap(), Arrays.asList(CONNECTOR2_NAME, CONNECTOR_NAME))); PowerMock.replayAll(); - Collection connectors = connectorsResource.listConnectors(FORWARD); - // Ordering isn't guaranteed, compare sets - assertEquals(new HashSet<>(Arrays.asList(CONNECTOR_NAME, CONNECTOR2_NAME)), new HashSet<>(connectors)); - + Collection connectors = connectorsResource.listConnectors(FORWARD, NULL_HEADERS); PowerMock.verifyAll(); } @@ -177,7 +176,7 @@ public void testListConnectorsNotSynced() throws Throwable { PowerMock.replayAll(); // throws - connectorsResource.listConnectors(FORWARD); + connectorsResource.listConnectors(FORWARD, NULL_HEADERS); } @Test @@ -191,7 +190,7 @@ public void testCreateConnector() throws Throwable { PowerMock.replayAll(); - connectorsResource.createConnector(FORWARD, body); + connectorsResource.createConnector(FORWARD, NULL_HEADERS, body); PowerMock.verifyAll(); } @@ -204,19 +203,57 @@ public void testCreateConnectorNotLeader() throws Throwable { herder.putConnectorConfig(EasyMock.eq(CONNECTOR_NAME), EasyMock.eq(body.config()), EasyMock.eq(false), EasyMock.capture(cb)); expectAndCallbackNotLeaderException(cb); // Should forward request - EasyMock.expect(RestClient.httpRequest(EasyMock.eq("http://leader:8083/connectors?forward=false"), EasyMock.eq("POST"), EasyMock.eq(body), EasyMock.anyObject(), EasyMock.anyObject(WorkerConfig.class))) + EasyMock.expect(RestClient.httpRequest(EasyMock.eq("http://leader:8083/connectors?forward=false"), EasyMock.eq("POST"), EasyMock.isNull(), EasyMock.eq(body), EasyMock.anyObject(), EasyMock.anyObject(WorkerConfig.class))) .andReturn(new RestClient.HttpResponse<>(201, new HashMap(), new ConnectorInfo(CONNECTOR_NAME, CONNECTOR_CONFIG, CONNECTOR_TASK_NAMES, ConnectorType.SOURCE))); PowerMock.replayAll(); - connectorsResource.createConnector(FORWARD, body); + connectorsResource.createConnector(FORWARD, NULL_HEADERS, body); PowerMock.verifyAll(); } + @Test + public void testCreateConnectorWithHeaderAuthorization() throws Throwable { + CreateConnectorRequest body = new CreateConnectorRequest(CONNECTOR_NAME, Collections.singletonMap(ConnectorConfig.NAME_CONFIG, CONNECTOR_NAME)); + final Capture>> cb = Capture.newInstance(); + HttpHeaders httpHeaders = EasyMock.mock(HttpHeaders.class); + EasyMock.expect(httpHeaders.getHeaderString("Authorization")).andReturn("Basic YWxhZGRpbjpvcGVuc2VzYW1l").times(1); + EasyMock.replay(httpHeaders); + herder.putConnectorConfig(EasyMock.eq(CONNECTOR_NAME), EasyMock.eq(body.config()), EasyMock.eq(false), EasyMock.capture(cb)); + expectAndCallbackResult(cb, new Herder.Created<>(true, new ConnectorInfo(CONNECTOR_NAME, CONNECTOR_CONFIG, + CONNECTOR_TASK_NAMES, ConnectorType.SOURCE))); + + PowerMock.replayAll(); + + connectorsResource.createConnector(FORWARD, httpHeaders, body); + + PowerMock.verifyAll(); + } + + + + @Test + public void testCreateConnectorWithoutHeaderAuthorization() throws Throwable { + CreateConnectorRequest body = new CreateConnectorRequest(CONNECTOR_NAME, Collections.singletonMap(ConnectorConfig.NAME_CONFIG, CONNECTOR_NAME)); + final Capture>> cb = Capture.newInstance(); + HttpHeaders httpHeaders = EasyMock.mock(HttpHeaders.class); + EasyMock.expect(httpHeaders.getHeaderString("Authorization")).andReturn(null).times(1); + EasyMock.replay(httpHeaders); + herder.putConnectorConfig(EasyMock.eq(CONNECTOR_NAME), EasyMock.eq(body.config()), EasyMock.eq(false), EasyMock.capture(cb)); + expectAndCallbackResult(cb, new Herder.Created<>(true, new ConnectorInfo(CONNECTOR_NAME, CONNECTOR_CONFIG, + CONNECTOR_TASK_NAMES, ConnectorType.SOURCE))); + + PowerMock.replayAll(); + + connectorsResource.createConnector(FORWARD, httpHeaders, body); + + PowerMock.verifyAll(); + } + @Test(expected = AlreadyExistsException.class) public void testCreateConnectorExists() throws Throwable { CreateConnectorRequest body = new CreateConnectorRequest(CONNECTOR_NAME, Collections.singletonMap(ConnectorConfig.NAME_CONFIG, CONNECTOR_NAME)); @@ -227,7 +264,7 @@ public void testCreateConnectorExists() throws Throwable { PowerMock.replayAll(); - connectorsResource.createConnector(FORWARD, body); + connectorsResource.createConnector(FORWARD, NULL_HEADERS, body); PowerMock.verifyAll(); } @@ -246,7 +283,7 @@ public void testCreateConnectorNameTrimWhitespaces() throws Throwable { PowerMock.replayAll(); - connectorsResource.createConnector(FORWARD, bodyIn); + connectorsResource.createConnector(FORWARD, NULL_HEADERS, bodyIn); PowerMock.verifyAll(); } @@ -265,7 +302,7 @@ public void testCreateConnectorNameAllWhitespaces() throws Throwable { PowerMock.replayAll(); - connectorsResource.createConnector(FORWARD, bodyIn); + connectorsResource.createConnector(FORWARD, NULL_HEADERS, bodyIn); PowerMock.verifyAll(); } @@ -284,7 +321,7 @@ public void testCreateConnectorNoName() throws Throwable { PowerMock.replayAll(); - connectorsResource.createConnector(FORWARD, bodyIn); + connectorsResource.createConnector(FORWARD, NULL_HEADERS, bodyIn); PowerMock.verifyAll(); } @@ -297,7 +334,7 @@ public void testDeleteConnector() throws Throwable { PowerMock.replayAll(); - connectorsResource.destroyConnector(CONNECTOR_NAME, FORWARD); + connectorsResource.destroyConnector(CONNECTOR_NAME, NULL_HEADERS, FORWARD); PowerMock.verifyAll(); } @@ -308,12 +345,12 @@ public void testDeleteConnectorNotLeader() throws Throwable { herder.deleteConnectorConfig(EasyMock.eq(CONNECTOR_NAME), EasyMock.capture(cb)); expectAndCallbackNotLeaderException(cb); // Should forward request - EasyMock.expect(RestClient.httpRequest("http://leader:8083/connectors/" + CONNECTOR_NAME + "?forward=false", "DELETE", null, null, null)) + EasyMock.expect(RestClient.httpRequest("http://leader:8083/connectors/" + CONNECTOR_NAME + "?forward=false", "DELETE", NULL_HEADERS, null, null, null)) .andReturn(new RestClient.HttpResponse<>(204, new HashMap(), null)); PowerMock.replayAll(); - connectorsResource.destroyConnector(CONNECTOR_NAME, FORWARD); + connectorsResource.destroyConnector(CONNECTOR_NAME, NULL_HEADERS, FORWARD); PowerMock.verifyAll(); } @@ -327,7 +364,7 @@ public void testDeleteConnectorNotFound() throws Throwable { PowerMock.replayAll(); - connectorsResource.destroyConnector(CONNECTOR_NAME, FORWARD); + connectorsResource.destroyConnector(CONNECTOR_NAME, NULL_HEADERS, FORWARD); PowerMock.verifyAll(); } @@ -341,7 +378,7 @@ public void testGetConnector() throws Throwable { PowerMock.replayAll(); - ConnectorInfo connInfo = connectorsResource.getConnector(CONNECTOR_NAME, FORWARD); + ConnectorInfo connInfo = connectorsResource.getConnector(CONNECTOR_NAME, NULL_HEADERS, FORWARD); assertEquals(new ConnectorInfo(CONNECTOR_NAME, CONNECTOR_CONFIG, CONNECTOR_TASK_NAMES, ConnectorType.SOURCE), connInfo); @@ -356,7 +393,7 @@ public void testGetConnectorConfig() throws Throwable { PowerMock.replayAll(); - Map connConfig = connectorsResource.getConnectorConfig(CONNECTOR_NAME, FORWARD); + Map connConfig = connectorsResource.getConnectorConfig(CONNECTOR_NAME, NULL_HEADERS, FORWARD); assertEquals(CONNECTOR_CONFIG, connConfig); PowerMock.verifyAll(); @@ -370,7 +407,7 @@ public void testGetConnectorConfigConnectorNotFound() throws Throwable { PowerMock.replayAll(); - connectorsResource.getConnectorConfig(CONNECTOR_NAME, FORWARD); + connectorsResource.getConnectorConfig(CONNECTOR_NAME, NULL_HEADERS, FORWARD); PowerMock.verifyAll(); } @@ -384,7 +421,7 @@ public void testPutConnectorConfig() throws Throwable { PowerMock.replayAll(); - connectorsResource.putConnectorConfig(CONNECTOR_NAME, FORWARD, CONNECTOR_CONFIG); + connectorsResource.putConnectorConfig(CONNECTOR_NAME, NULL_HEADERS, FORWARD, CONNECTOR_CONFIG); PowerMock.verifyAll(); } @@ -400,7 +437,7 @@ public void testCreateConnectorWithSpecialCharsInName() throws Throwable { PowerMock.replayAll(); - String rspLocation = connectorsResource.createConnector(FORWARD, body).getLocation().toString(); + String rspLocation = connectorsResource.createConnector(FORWARD, NULL_HEADERS, body).getLocation().toString(); String decoded = new URI(rspLocation).getPath(); Assert.assertEquals("/connectors/" + CONNECTOR_NAME_SPECIAL_CHARS, decoded); @@ -418,7 +455,7 @@ public void testCreateConnectorWithControlSequenceInName() throws Throwable { PowerMock.replayAll(); - String rspLocation = connectorsResource.createConnector(FORWARD, body).getLocation().toString(); + String rspLocation = connectorsResource.createConnector(FORWARD, NULL_HEADERS, body).getLocation().toString(); String decoded = new URI(rspLocation).getPath(); Assert.assertEquals("/connectors/" + CONNECTOR_NAME_CONTROL_SEQUENCES1, decoded); @@ -435,7 +472,7 @@ public void testPutConnectorConfigWithSpecialCharsInName() throws Throwable { PowerMock.replayAll(); - String rspLocation = connectorsResource.putConnectorConfig(CONNECTOR_NAME_SPECIAL_CHARS, FORWARD, CONNECTOR_CONFIG_SPECIAL_CHARS).getLocation().toString(); + String rspLocation = connectorsResource.putConnectorConfig(CONNECTOR_NAME_SPECIAL_CHARS, NULL_HEADERS, FORWARD, CONNECTOR_CONFIG_SPECIAL_CHARS).getLocation().toString(); String decoded = new URI(rspLocation).getPath(); Assert.assertEquals("/connectors/" + CONNECTOR_NAME_SPECIAL_CHARS, decoded); @@ -452,7 +489,7 @@ public void testPutConnectorConfigWithControlSequenceInName() throws Throwable { PowerMock.replayAll(); - String rspLocation = connectorsResource.putConnectorConfig(CONNECTOR_NAME_CONTROL_SEQUENCES1, FORWARD, CONNECTOR_CONFIG_CONTROL_SEQUENCES).getLocation().toString(); + String rspLocation = connectorsResource.putConnectorConfig(CONNECTOR_NAME_CONTROL_SEQUENCES1, NULL_HEADERS, FORWARD, CONNECTOR_CONFIG_CONTROL_SEQUENCES).getLocation().toString(); String decoded = new URI(rspLocation).getPath(); Assert.assertEquals("/connectors/" + CONNECTOR_NAME_CONTROL_SEQUENCES1, decoded); @@ -463,7 +500,7 @@ public void testPutConnectorConfigWithControlSequenceInName() throws Throwable { public void testPutConnectorConfigNameMismatch() throws Throwable { Map connConfig = new HashMap<>(CONNECTOR_CONFIG); connConfig.put(ConnectorConfig.NAME_CONFIG, "mismatched-name"); - connectorsResource.putConnectorConfig(CONNECTOR_NAME, FORWARD, connConfig); + connectorsResource.putConnectorConfig(CONNECTOR_NAME, NULL_HEADERS, FORWARD, connConfig); } @Test(expected = BadRequestException.class) @@ -471,7 +508,7 @@ public void testCreateConnectorConfigNameMismatch() throws Throwable { Map connConfig = new HashMap<>(); connConfig.put(ConnectorConfig.NAME_CONFIG, "mismatched-name"); CreateConnectorRequest request = new CreateConnectorRequest(CONNECTOR_NAME, connConfig); - connectorsResource.createConnector(FORWARD, request); + connectorsResource.createConnector(FORWARD, NULL_HEADERS, request); } @Test @@ -482,7 +519,7 @@ public void testGetConnectorTaskConfigs() throws Throwable { PowerMock.replayAll(); - List taskInfos = connectorsResource.getTaskConfigs(CONNECTOR_NAME, FORWARD); + List taskInfos = connectorsResource.getTaskConfigs(CONNECTOR_NAME, NULL_HEADERS, FORWARD); assertEquals(TASK_INFOS, taskInfos); PowerMock.verifyAll(); @@ -496,7 +533,7 @@ public void testGetConnectorTaskConfigsConnectorNotFound() throws Throwable { PowerMock.replayAll(); - connectorsResource.getTaskConfigs(CONNECTOR_NAME, FORWARD); + connectorsResource.getTaskConfigs(CONNECTOR_NAME, NULL_HEADERS, FORWARD); PowerMock.verifyAll(); } @@ -509,7 +546,7 @@ public void testPutConnectorTaskConfigs() throws Throwable { PowerMock.replayAll(); - connectorsResource.putTaskConfigs(CONNECTOR_NAME, FORWARD, TASK_CONFIGS); + connectorsResource.putTaskConfigs(CONNECTOR_NAME, NULL_HEADERS, FORWARD, TASK_CONFIGS); PowerMock.verifyAll(); } @@ -522,7 +559,7 @@ public void testPutConnectorTaskConfigsConnectorNotFound() throws Throwable { PowerMock.replayAll(); - connectorsResource.putTaskConfigs(CONNECTOR_NAME, FORWARD, TASK_CONFIGS); + connectorsResource.putTaskConfigs(CONNECTOR_NAME, NULL_HEADERS, FORWARD, TASK_CONFIGS); PowerMock.verifyAll(); } @@ -535,7 +572,7 @@ public void testRestartConnectorNotFound() throws Throwable { PowerMock.replayAll(); - connectorsResource.restartConnector(CONNECTOR_NAME, FORWARD); + connectorsResource.restartConnector(CONNECTOR_NAME, NULL_HEADERS, FORWARD); PowerMock.verifyAll(); } @@ -547,12 +584,12 @@ public void testRestartConnectorLeaderRedirect() throws Throwable { expectAndCallbackNotLeaderException(cb); EasyMock.expect(RestClient.httpRequest(EasyMock.eq("http://leader:8083/connectors/" + CONNECTOR_NAME + "/restart?forward=true"), - EasyMock.eq("POST"), EasyMock.isNull(), EasyMock.anyObject(), EasyMock.anyObject(WorkerConfig.class))) + EasyMock.eq("POST"), EasyMock.isNull(), EasyMock.isNull(), EasyMock.anyObject(), EasyMock.anyObject(WorkerConfig.class))) .andReturn(new RestClient.HttpResponse<>(202, new HashMap(), null)); PowerMock.replayAll(); - connectorsResource.restartConnector(CONNECTOR_NAME, null); + connectorsResource.restartConnector(CONNECTOR_NAME, NULL_HEADERS, null); PowerMock.verifyAll(); } @@ -565,12 +602,12 @@ public void testRestartConnectorOwnerRedirect() throws Throwable { expectAndCallbackException(cb, new NotAssignedException("not owner test", ownerUrl)); EasyMock.expect(RestClient.httpRequest(EasyMock.eq("http://owner:8083/connectors/" + CONNECTOR_NAME + "/restart?forward=false"), - EasyMock.eq("POST"), EasyMock.isNull(), EasyMock.anyObject(), EasyMock.anyObject(WorkerConfig.class))) + EasyMock.eq("POST"), EasyMock.isNull(), EasyMock.isNull(), EasyMock.anyObject(), EasyMock.anyObject(WorkerConfig.class))) .andReturn(new RestClient.HttpResponse<>(202, new HashMap(), null)); PowerMock.replayAll(); - connectorsResource.restartConnector(CONNECTOR_NAME, true); + connectorsResource.restartConnector(CONNECTOR_NAME, NULL_HEADERS, true); PowerMock.verifyAll(); } @@ -584,7 +621,7 @@ public void testRestartTaskNotFound() throws Throwable { PowerMock.replayAll(); - connectorsResource.restartTask(CONNECTOR_NAME, 0, FORWARD); + connectorsResource.restartTask(CONNECTOR_NAME, 0, NULL_HEADERS, FORWARD); PowerMock.verifyAll(); } @@ -598,12 +635,12 @@ public void testRestartTaskLeaderRedirect() throws Throwable { expectAndCallbackNotLeaderException(cb); EasyMock.expect(RestClient.httpRequest(EasyMock.eq("http://leader:8083/connectors/" + CONNECTOR_NAME + "/tasks/0/restart?forward=true"), - EasyMock.eq("POST"), EasyMock.isNull(), EasyMock.anyObject(), EasyMock.anyObject(WorkerConfig.class))) + EasyMock.eq("POST"), EasyMock.isNull(), EasyMock.isNull(), EasyMock.anyObject(), EasyMock.anyObject(WorkerConfig.class))) .andReturn(new RestClient.HttpResponse<>(202, new HashMap(), null)); PowerMock.replayAll(); - connectorsResource.restartTask(CONNECTOR_NAME, 0, null); + connectorsResource.restartTask(CONNECTOR_NAME, 0, NULL_HEADERS, null); PowerMock.verifyAll(); } @@ -618,12 +655,12 @@ public void testRestartTaskOwnerRedirect() throws Throwable { expectAndCallbackException(cb, new NotAssignedException("not owner test", ownerUrl)); EasyMock.expect(RestClient.httpRequest(EasyMock.eq("http://owner:8083/connectors/" + CONNECTOR_NAME + "/tasks/0/restart?forward=false"), - EasyMock.eq("POST"), EasyMock.isNull(), EasyMock.anyObject(), EasyMock.anyObject(WorkerConfig.class))) + EasyMock.eq("POST"), EasyMock.isNull(), EasyMock.isNull(), EasyMock.anyObject(), EasyMock.anyObject(WorkerConfig.class))) .andReturn(new RestClient.HttpResponse<>(202, new HashMap(), null)); PowerMock.replayAll(); - connectorsResource.restartTask(CONNECTOR_NAME, 0, true); + connectorsResource.restartTask(CONNECTOR_NAME, 0, NULL_HEADERS, true); PowerMock.verifyAll(); } diff --git a/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/standalone/StandaloneHerderTest.java b/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/standalone/StandaloneHerderTest.java index 5372a3a27a5c0..a23ee10ea480f 100644 --- a/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/standalone/StandaloneHerderTest.java +++ b/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/standalone/StandaloneHerderTest.java @@ -36,6 +36,7 @@ import org.apache.kafka.connect.runtime.TaskConfig; import org.apache.kafka.connect.runtime.TaskStatus; import org.apache.kafka.connect.runtime.Worker; +import org.apache.kafka.connect.runtime.WorkerConfigTransformer; import org.apache.kafka.connect.runtime.WorkerConnector; import org.apache.kafka.connect.runtime.distributed.ClusterConfigState; import org.apache.kafka.connect.runtime.isolation.DelegatingClassLoader; @@ -101,6 +102,7 @@ private enum SourceSink { private Connector connector; @Mock protected Worker worker; + @Mock protected WorkerConfigTransformer transformer; @Mock private Plugins plugins; @Mock private PluginClassLoader pluginLoader; @@ -113,12 +115,14 @@ private enum SourceSink { public void setup() { worker = PowerMock.createMock(Worker.class); herder = PowerMock.createPartialMock(StandaloneHerder.class, new String[]{"connectorTypeForClass"}, - worker, WORKER_ID, KAFKA_CLUSTER_ID, statusBackingStore, new MemoryConfigBackingStore()); + worker, WORKER_ID, KAFKA_CLUSTER_ID, statusBackingStore, new MemoryConfigBackingStore(transformer)); plugins = PowerMock.createMock(Plugins.class); pluginLoader = PowerMock.createMock(PluginClassLoader.class); delegatingLoader = PowerMock.createMock(DelegatingClassLoader.class); PowerMock.mockStatic(Plugins.class); PowerMock.mockStatic(WorkerConnector.class); + Capture> configCapture = Capture.newInstance(); + EasyMock.expect(transformer.transform(EasyMock.eq(CONNECTOR_NAME), EasyMock.capture(configCapture))).andAnswer(configCapture::getValue).anyTimes(); } @Test @@ -146,6 +150,9 @@ public void testCreateConnectorFailedBasicValidation() throws Exception { config.remove(ConnectorConfig.NAME_CONFIG); Connector connectorMock = PowerMock.createMock(SourceConnector.class); + EasyMock.expect(worker.configTransformer()).andReturn(transformer).times(2); + final Capture> configCapture = EasyMock.newCapture(); + EasyMock.expect(transformer.transform(EasyMock.capture(configCapture))).andAnswer(configCapture::getValue); EasyMock.expect(worker.getPlugins()).andReturn(plugins).times(3); EasyMock.expect(plugins.compareAndSwapLoaders(connectorMock)).andReturn(delegatingLoader); EasyMock.expect(plugins.newConnector(EasyMock.anyString())).andReturn(connectorMock); @@ -171,6 +178,9 @@ public void testCreateConnectorFailedCustomValidation() throws Exception { connector = PowerMock.createMock(BogusSourceConnector.class); Connector connectorMock = PowerMock.createMock(SourceConnector.class); + EasyMock.expect(worker.configTransformer()).andReturn(transformer).times(2); + final Capture> configCapture = EasyMock.newCapture(); + EasyMock.expect(transformer.transform(EasyMock.capture(configCapture))).andAnswer(configCapture::getValue); EasyMock.expect(worker.getPlugins()).andReturn(plugins).times(3); EasyMock.expect(plugins.compareAndSwapLoaders(connectorMock)).andReturn(delegatingLoader); EasyMock.expect(plugins.newConnector(EasyMock.anyString())).andReturn(connectorMock); @@ -205,6 +215,9 @@ public void testCreateConnectorAlreadyExists() throws Exception { Connector connectorMock = PowerMock.createMock(SourceConnector.class); expectConfigValidation(connectorMock, true, config, config); + EasyMock.expect(worker.configTransformer()).andReturn(transformer).times(2); + final Capture> configCapture = EasyMock.newCapture(); + EasyMock.expect(transformer.transform(EasyMock.capture(configCapture))).andAnswer(configCapture::getValue); EasyMock.expect(worker.getPlugins()).andReturn(plugins).times(2); EasyMock.expect(plugins.compareAndSwapLoaders(connectorMock)).andReturn(delegatingLoader); // No new connector is created @@ -346,7 +359,8 @@ public void testRestartTask() throws Exception { Collections.singletonMap(CONNECTOR_NAME, connectorConfig), Collections.singletonMap(CONNECTOR_NAME, TargetState.STARTED), Collections.singletonMap(taskId, taskConfig(SourceSink.SOURCE)), - new HashSet<>()); + new HashSet<>(), + transformer); worker.startTask(taskId, configState, connectorConfig, taskConfig(SourceSink.SOURCE), herder, TargetState.STARTED); EasyMock.expectLastCall().andReturn(true); @@ -379,7 +393,8 @@ public void testRestartTaskFailureOnStart() throws Exception { Collections.singletonMap(CONNECTOR_NAME, connectorConfig), Collections.singletonMap(CONNECTOR_NAME, TargetState.STARTED), Collections.singletonMap(new ConnectorTaskId(CONNECTOR_NAME, 0), taskConfig(SourceSink.SOURCE)), - new HashSet<>()); + new HashSet<>(), + transformer); worker.startTask(taskId, configState, connectorConfig, taskConfig(SourceSink.SOURCE), herder, TargetState.STARTED); EasyMock.expectLastCall().andReturn(false); @@ -447,7 +462,6 @@ public void testAccessors() throws Exception { // Create connector connector = PowerMock.createMock(BogusSourceConnector.class); expectAdd(SourceSink.SOURCE); - Connector connectorMock = PowerMock.createMock(SourceConnector.class); expectConfigValidation(connector, true, connConfig); // Validate accessors with 1 connector @@ -474,6 +488,13 @@ public void testAccessors() throws Exception { herder.taskConfigs(CONNECTOR_NAME, taskConfigsCb); herder.putConnectorConfig(CONNECTOR_NAME, connConfig, false, createCallback); + + EasyMock.reset(transformer); + EasyMock.expect(transformer.transform(EasyMock.eq(CONNECTOR_NAME), EasyMock.anyObject())) + .andThrow(new AssertionError("Config transformation should not occur when requesting connector or task info")) + .anyTimes(); + EasyMock.replay(transformer); + herder.connectors(listConnectorsCb); herder.connectorInfo(CONNECTOR_NAME, connectorInfoCb); herder.connectorConfig(CONNECTOR_NAME, connectorConfigCb); @@ -565,6 +586,9 @@ public void testCorruptConfig() { ); ConfigDef configDef = new ConfigDef(); configDef.define(key, ConfigDef.Type.STRING, ConfigDef.Importance.HIGH, ""); + EasyMock.expect(worker.configTransformer()).andReturn(transformer).times(2); + final Capture> configCapture = EasyMock.newCapture(); + EasyMock.expect(transformer.transform(EasyMock.capture(configCapture))).andAnswer(configCapture::getValue); EasyMock.expect(worker.getPlugins()).andReturn(plugins).times(3); EasyMock.expect(plugins.compareAndSwapLoaders(connectorMock)).andReturn(delegatingLoader); EasyMock.expect(worker.getPlugins()).andStubReturn(plugins); @@ -590,8 +614,7 @@ public void testCorruptConfig() { PowerMock.verifyAll(); } - private void expectAdd(SourceSink sourceSink) throws Exception { - + private void expectAdd(SourceSink sourceSink) { Map connectorProps = connectorConfig(sourceSink); ConnectorConfig connConfig = sourceSink == SourceSink.SOURCE ? new SourceConnectorConfig(plugins, connectorProps) : @@ -620,7 +643,8 @@ private void expectAdd(SourceSink sourceSink) throws Exception { Collections.singletonMap(CONNECTOR_NAME, connectorConfig(sourceSink)), Collections.singletonMap(CONNECTOR_NAME, TargetState.STARTED), Collections.singletonMap(new ConnectorTaskId(CONNECTOR_NAME, 0), generatedTaskProps), - new HashSet<>()); + new HashSet<>(), + transformer); worker.startTask(new ConnectorTaskId(CONNECTOR_NAME, 0), configState, connectorConfig(sourceSink), generatedTaskProps, herder, TargetState.STARTED); EasyMock.expectLastCall().andReturn(true); @@ -672,6 +696,9 @@ private void expectConfigValidation( Map... configs ) { // config validation + EasyMock.expect(worker.configTransformer()).andReturn(transformer).times(2); + final Capture> configCapture = EasyMock.newCapture(); + EasyMock.expect(transformer.transform(EasyMock.capture(configCapture))).andAnswer(configCapture::getValue); EasyMock.expect(worker.getPlugins()).andReturn(plugins).times(3); EasyMock.expect(plugins.compareAndSwapLoaders(connectorMock)).andReturn(delegatingLoader); if (shouldCreateConnector) { diff --git a/connect/runtime/src/test/java/org/apache/kafka/connect/storage/FileOffsetBackingStoreTest.java b/connect/runtime/src/test/java/org/apache/kafka/connect/storage/FileOffsetBackingStoreTest.java index c6b61b4d2045c..7c87bf9a62f3c 100644 --- a/connect/runtime/src/test/java/org/apache/kafka/connect/storage/FileOffsetBackingStoreTest.java +++ b/connect/runtime/src/test/java/org/apache/kafka/connect/storage/FileOffsetBackingStoreTest.java @@ -70,12 +70,11 @@ public void teardown() { @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(); + Map values = store.get(Arrays.asList(buffer("key"), buffer("bad"))).get(); assertEquals(buffer("value"), values.get(buffer("key"))); assertEquals(null, values.get(buffer("bad"))); @@ -85,7 +84,6 @@ public void testGetSet() throws Exception { @Test public void testSaveRestore() throws Exception { Callback setCallback = expectSuccessfulSetCallback(); - Callback> getCallback = expectSuccessfulGetCallback(); PowerMock.replayAll(); store.set(firstSet, setCallback).get(); @@ -95,7 +93,7 @@ public void testSaveRestore() throws Exception { FileOffsetBackingStore restore = new FileOffsetBackingStore(); restore.configure(config); restore.start(); - Map values = restore.get(Arrays.asList(buffer("key")), getCallback).get(); + Map values = restore.get(Arrays.asList(buffer("key"))).get(); assertEquals(buffer("value"), values.get(buffer("key"))); PowerMock.verifyAll(); @@ -112,12 +110,4 @@ private Callback expectSuccessfulSetCallback() { 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/connect/runtime/src/test/java/org/apache/kafka/connect/storage/KafkaOffsetBackingStoreTest.java b/connect/runtime/src/test/java/org/apache/kafka/connect/storage/KafkaOffsetBackingStoreTest.java index 8cd2f0b0560aa..97f59b0a24e86 100644 --- a/connect/runtime/src/test/java/org/apache/kafka/connect/storage/KafkaOffsetBackingStoreTest.java +++ b/connect/runtime/src/test/java/org/apache/kafka/connect/storage/KafkaOffsetBackingStoreTest.java @@ -220,17 +220,10 @@ public Object answer() throws Throwable { store.start(); // Getting from empty store should return nulls - 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()); + Map offsets = store.get(Arrays.asList(TP0_KEY, TP1_KEY)).get(10000, TimeUnit.MILLISECONDS); + // Since we didn't read them yet, these will be null + assertNull(offsets.get(TP0_KEY)); + assertNull(offsets.get(TP1_KEY)); // Set some offsets Map toSet = new HashMap<>(); @@ -253,28 +246,14 @@ public void onCompletion(Throwable error, Void result) { assertTrue(invoked.get()); // Getting data should read to end of our published data and return it - final AtomicBoolean secondGetInvokedAndPassed = new AtomicBoolean(false); - store.get(Arrays.asList(TP0_KEY, TP1_KEY), new Callback>() { - @Override - public void onCompletion(Throwable error, Map result) { - assertEquals(TP0_VALUE, result.get(TP0_KEY)); - assertEquals(TP1_VALUE, result.get(TP1_KEY)); - secondGetInvokedAndPassed.set(true); - } - }).get(10000, TimeUnit.MILLISECONDS); - assertTrue(secondGetInvokedAndPassed.get()); + offsets = store.get(Arrays.asList(TP0_KEY, TP1_KEY)).get(10000, TimeUnit.MILLISECONDS); + assertEquals(TP0_VALUE, offsets.get(TP0_KEY)); + assertEquals(TP1_VALUE, offsets.get(TP1_KEY)); // Getting data should read to end of our published data and return it - final AtomicBoolean thirdGetInvokedAndPassed = 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)); - thirdGetInvokedAndPassed.set(true); - } - }).get(10000, TimeUnit.MILLISECONDS); - assertTrue(thirdGetInvokedAndPassed.get()); + offsets = store.get(Arrays.asList(TP0_KEY, TP1_KEY)).get(10000, TimeUnit.MILLISECONDS); + assertEquals(TP0_VALUE_NEW, offsets.get(TP0_KEY)); + assertEquals(TP1_VALUE_NEW, offsets.get(TP1_KEY)); store.stop(); @@ -335,16 +314,9 @@ public void onCompletion(Throwable error, Void result) { assertTrue(invoked.get()); // Getting data should read to end of our published data and return it - final AtomicBoolean secondGetInvokedAndPassed = new AtomicBoolean(false); - store.get(Arrays.asList(null, TP1_KEY), new Callback>() { - @Override - public void onCompletion(Throwable error, Map result) { - assertEquals(TP0_VALUE, result.get(null)); - assertNull(result.get(TP1_KEY)); - secondGetInvokedAndPassed.set(true); - } - }).get(10000, TimeUnit.MILLISECONDS); - assertTrue(secondGetInvokedAndPassed.get()); + Map offsets = store.get(Arrays.asList(null, TP1_KEY)).get(10000, TimeUnit.MILLISECONDS); + assertEquals(TP0_VALUE, offsets.get(null)); + assertNull(offsets.get(TP1_KEY)); store.stop(); diff --git a/connect/runtime/src/test/java/org/apache/kafka/connect/util/ConvertingFutureCallbackTest.java b/connect/runtime/src/test/java/org/apache/kafka/connect/util/ConvertingFutureCallbackTest.java new file mode 100644 index 0000000000000..9535003c863c3 --- /dev/null +++ b/connect/runtime/src/test/java/org/apache/kafka/connect/util/ConvertingFutureCallbackTest.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.connect.util; + +import org.junit.Before; +import org.junit.Test; + +import java.util.concurrent.CancellationException; +import java.util.concurrent.CountDownLatch; +import java.util.concurrent.ExecutionException; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.Executors; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.TimeoutException; +import java.util.concurrent.atomic.AtomicInteger; +import java.util.concurrent.atomic.AtomicReference; + +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 ConvertingFutureCallbackTest { + + private ExecutorService executor; + + @Before + public void setup() { + executor = Executors.newSingleThreadExecutor(); + } + + @Test + public void shouldConvertBeforeGetOnSuccessfulCompletion() throws Exception { + final Object expectedConversion = new Object(); + TestConvertingFutureCallback testCallback = new TestConvertingFutureCallback(); + testCallback.onCompletion(null, expectedConversion); + assertEquals(1, testCallback.numberOfConversions()); + assertEquals(expectedConversion, testCallback.get()); + } + + @Test + public void shouldConvertOnlyOnceBeforeGetOnSuccessfulCompletion() throws Exception { + final Object expectedConversion = new Object(); + TestConvertingFutureCallback testCallback = new TestConvertingFutureCallback(); + testCallback.onCompletion(null, expectedConversion); + testCallback.onCompletion(null, 69); + testCallback.cancel(true); + testCallback.onCompletion(new RuntimeException(), null); + assertEquals(1, testCallback.numberOfConversions()); + assertEquals(expectedConversion, testCallback.get()); + } + + @Test + public void shouldNotConvertBeforeGetOnFailedCompletion() throws Exception { + final Throwable expectedError = new Throwable(); + TestConvertingFutureCallback testCallback = new TestConvertingFutureCallback(); + testCallback.onCompletion(expectedError, null); + assertEquals(0, testCallback.numberOfConversions()); + try { + testCallback.get(); + fail("Expected ExecutionException"); + } catch (ExecutionException e) { + assertEquals(expectedError, e.getCause()); + } + } + + @Test + public void shouldRecordOnlyFirstErrorBeforeGetOnFailedCompletion() throws Exception { + final Throwable expectedError = new Throwable(); + TestConvertingFutureCallback testCallback = new TestConvertingFutureCallback(); + testCallback.onCompletion(expectedError, null); + testCallback.onCompletion(new RuntimeException(), null); + testCallback.cancel(true); + testCallback.onCompletion(null, "420"); + assertEquals(0, testCallback.numberOfConversions()); + try { + testCallback.get(); + fail("Expected ExecutionException"); + } catch (ExecutionException e) { + assertEquals(expectedError, e.getCause()); + } + } + + @Test(expected = CancellationException.class) + public void shouldCancelBeforeGetIfMayCancelWhileRunning() throws Exception { + TestConvertingFutureCallback testCallback = new TestConvertingFutureCallback(); + assertTrue(testCallback.cancel(true)); + testCallback.get(); + } + + @Test + public void shouldBlockUntilSuccessfulCompletion() throws Exception { + AtomicReference testThreadException = new AtomicReference<>(); + TestConvertingFutureCallback testCallback = new TestConvertingFutureCallback(); + final Object expectedConversion = new Object(); + executor.submit(() -> { + try { + testCallback.waitForGet(); + testCallback.onCompletion(null, expectedConversion); + } catch (Exception e) { + testThreadException.compareAndSet(null, e); + } + }); + assertFalse(testCallback.isDone()); + assertEquals(expectedConversion, testCallback.get()); + assertEquals(1, testCallback.numberOfConversions()); + assertTrue(testCallback.isDone()); + if (testThreadException.get() != null) { + throw testThreadException.get(); + } + } + + @Test + public void shouldBlockUntilFailedCompletion() throws Exception { + AtomicReference testThreadException = new AtomicReference<>(); + TestConvertingFutureCallback testCallback = new TestConvertingFutureCallback(); + final Throwable expectedError = new Throwable(); + executor.submit(() -> { + try { + testCallback.waitForGet(); + testCallback.onCompletion(expectedError, null); + } catch (Exception e) { + testThreadException.compareAndSet(null, e); + } + }); + assertFalse(testCallback.isDone()); + try { + testCallback.get(); + fail("Expected ExecutionException"); + } catch (ExecutionException e) { + assertEquals(expectedError, e.getCause()); + } + assertEquals(0, testCallback.numberOfConversions()); + assertTrue(testCallback.isDone()); + if (testThreadException.get() != null) { + throw testThreadException.get(); + } + } + + @Test(expected = CancellationException.class) + public void shouldBlockUntilCancellation() throws Exception { + AtomicReference testThreadException = new AtomicReference<>(); + TestConvertingFutureCallback testCallback = new TestConvertingFutureCallback(); + executor.submit(() -> { + try { + testCallback.waitForGet(); + testCallback.cancel(true); + } catch (Exception e) { + testThreadException.compareAndSet(null, e); + } + }); + assertFalse(testCallback.isDone()); + testCallback.get(); + if (testThreadException.get() != null) { + throw testThreadException.get(); + } + } + + @Test + public void shouldNotCancelIfMayNotCancelWhileRunning() throws Exception { + AtomicReference testThreadException = new AtomicReference<>(); + TestConvertingFutureCallback testCallback = new TestConvertingFutureCallback(); + final Object expectedConversion = new Object(); + executor.submit(() -> { + try { + testCallback.waitForCancel(); + testCallback.onCompletion(null, expectedConversion); + } catch (Exception e) { + testThreadException.compareAndSet(null, e); + } + }); + assertFalse(testCallback.isCancelled()); + assertFalse(testCallback.isDone()); + testCallback.cancel(false); + assertFalse(testCallback.isCancelled()); + assertTrue(testCallback.isDone()); + assertEquals(expectedConversion, testCallback.get()); + assertEquals(1, testCallback.numberOfConversions()); + if (testThreadException.get() != null) { + throw testThreadException.get(); + } + } + + protected static class TestConvertingFutureCallback extends ConvertingFutureCallback { + private AtomicInteger numberOfConversions = new AtomicInteger(); + private CountDownLatch getInvoked = new CountDownLatch(1); + private CountDownLatch cancelInvoked = new CountDownLatch(1); + + public int numberOfConversions() { + return numberOfConversions.get(); + } + + public void waitForGet() throws InterruptedException { + getInvoked.await(); + } + + public void waitForCancel() throws InterruptedException { + cancelInvoked.await(); + } + + @Override + public Object convert(Object result) { + numberOfConversions.incrementAndGet(); + return result; + } + + @Override + public Object get() throws InterruptedException, ExecutionException { + getInvoked.countDown(); + return super.get(); + } + + @Override + public Object get( + long duration, + TimeUnit unit + ) throws InterruptedException, ExecutionException, TimeoutException { + getInvoked.countDown(); + return super.get(duration, unit); + } + + @Override + public boolean cancel(boolean mayInterruptIfRunning) { + cancelInvoked.countDown(); + return super.cancel(mayInterruptIfRunning); + } + } +} diff --git a/connect/runtime/src/test/java/org/apache/kafka/connect/util/KafkaBasedLogTest.java b/connect/runtime/src/test/java/org/apache/kafka/connect/util/KafkaBasedLogTest.java index 6d5efe8a3783d..180555010d1f4 100644 --- a/connect/runtime/src/test/java/org/apache/kafka/connect/util/KafkaBasedLogTest.java +++ b/connect/runtime/src/test/java/org/apache/kafka/connect/util/KafkaBasedLogTest.java @@ -30,6 +30,7 @@ import org.apache.kafka.common.PartitionInfo; import org.apache.kafka.common.TopicPartition; import org.apache.kafka.common.errors.LeaderNotAvailableException; +import org.apache.kafka.common.errors.TimeoutException; import org.apache.kafka.common.errors.WakeupException; import org.apache.kafka.common.protocol.Errors; import org.apache.kafka.common.record.TimestampType; @@ -369,7 +370,7 @@ public void run() { } @Test - public void testConsumerError() throws Exception { + public void testPollConsumerError() throws Exception { expectStart(); expectStop(); @@ -387,7 +388,7 @@ public void run() { consumer.schedulePollTask(new Runnable() { @Override public void run() { - consumer.setException(Errors.COORDINATOR_NOT_AVAILABLE.exception()); + consumer.setPollException(Errors.COORDINATOR_NOT_AVAILABLE.exception()); } }); @@ -422,6 +423,77 @@ public void run() { PowerMock.verifyAll(); } + @Test + public void testGetOffsetsConsumerErrorOnReadToEnd() throws Exception { + expectStart(); + + // Producer flushes when read to log end is called + producer.flush(); + PowerMock.expectLastCall(); + + expectStop(); + + PowerMock.replayAll(); + final CountDownLatch finishedLatch = new CountDownLatch(1); + Map endOffsets = new HashMap<>(); + endOffsets.put(TP0, 0L); + endOffsets.put(TP1, 0L); + consumer.updateEndOffsets(endOffsets); + store.start(); + final AtomicBoolean getInvoked = new AtomicBoolean(false); + final FutureCallback readEndFutureCallback = new FutureCallback<>(new Callback() { + @Override + public void onCompletion(Throwable error, Void result) { + getInvoked.set(true); + } + }); + consumer.schedulePollTask(new Runnable() { + @Override + public void run() { + // Once we're synchronized in a poll, start the read to end and schedule the exact set of poll events + // that should follow. This readToEnd call will immediately wakeup this consumer.poll() call without + // returning any data. + Map newEndOffsets = new HashMap<>(); + newEndOffsets.put(TP0, 1L); + newEndOffsets.put(TP1, 1L); + consumer.updateEndOffsets(newEndOffsets); + // Set exception to occur when getting offsets to read log to end. It'll be caught in the work thread, + // which will retry and eventually get the correct offsets and read log to end. + consumer.setOffsetsException(new TimeoutException("Failed to get offsets by times")); + store.readToEnd(readEndFutureCallback); + + // Should keep polling until it reaches current log end offset for all partitions + consumer.scheduleNopPollTask(); + consumer.scheduleNopPollTask(); + consumer.schedulePollTask(new Runnable() { + @Override + public void run() { + consumer.addRecord(new ConsumerRecord<>(TOPIC, 0, 0, 0L, TimestampType.CREATE_TIME, 0L, 0, 0, TP0_KEY, TP0_VALUE)); + consumer.addRecord(new ConsumerRecord<>(TOPIC, 1, 0, 0L, TimestampType.CREATE_TIME, 0L, 0, 0, TP0_KEY, TP0_VALUE_NEW)); + } + }); + + consumer.schedulePollTask(new Runnable() { + @Override + public void run() { + finishedLatch.countDown(); + } + }); + } + }); + readEndFutureCallback.get(10000, TimeUnit.MILLISECONDS); + assertTrue(getInvoked.get()); + assertTrue(finishedLatch.await(10000, TimeUnit.MILLISECONDS)); + assertEquals(CONSUMER_ASSIGNMENT, consumer.assignment()); + assertEquals(1L, consumer.position(TP0)); + + store.stop(); + + assertFalse(Whitebox.getInternalState(store, "thread").isAlive()); + assertTrue(consumer.closed()); + PowerMock.verifyAll(); + } + @Test public void testProducerError() throws Exception { expectStart(); diff --git a/connect/runtime/src/test/java/org/apache/kafka/connect/util/TopicAdminTest.java b/connect/runtime/src/test/java/org/apache/kafka/connect/util/TopicAdminTest.java index 5b1e15535598b..7b6f47ca2cb2e 100644 --- a/connect/runtime/src/test/java/org/apache/kafka/connect/util/TopicAdminTest.java +++ b/connect/runtime/src/test/java/org/apache/kafka/connect/util/TopicAdminTest.java @@ -17,8 +17,8 @@ package org.apache.kafka.connect.util; import org.apache.kafka.clients.NodeApiVersions; -import org.apache.kafka.clients.admin.MockAdminClient; import org.apache.kafka.clients.admin.AdminClientUnitTestEnv; +import org.apache.kafka.clients.admin.MockAdminClient; import org.apache.kafka.clients.admin.NewTopic; import org.apache.kafka.common.Cluster; import org.apache.kafka.common.Node; @@ -72,6 +72,19 @@ public void returnNullWithClusterAuthorizationFailure() { } } + @Test + public void returnNullWithTopicAuthorizationFailure() { + final NewTopic newTopic = TopicAdmin.defineTopic("myTopic").partitions(1).compacted().build(); + Cluster cluster = createCluster(1); + try (AdminClientUnitTestEnv env = new AdminClientUnitTestEnv(cluster)) { + env.kafkaClient().setNode(cluster.nodes().iterator().next()); + env.kafkaClient().prepareResponse(createTopicResponseWithTopicAuthorizationException(newTopic)); + TopicAdmin admin = new TopicAdmin(null, env.adminClient()); + boolean created = admin.createTopic(newTopic); + assertFalse(created); + } + } + @Test public void shouldNotCreateTopicWhenItAlreadyExists() { NewTopic newTopic = TopicAdmin.defineTopic("myTopic").partitions(1).compacted().build(); @@ -136,6 +149,10 @@ private CreateTopicsResponse createTopicResponseWithClusterAuthorizationExceptio return createTopicResponse(new ApiError(Errors.CLUSTER_AUTHORIZATION_FAILED, "Not authorized to create topic(s)"), topics); } + private CreateTopicsResponse createTopicResponseWithTopicAuthorizationException(NewTopic... topics) { + return createTopicResponse(new ApiError(Errors.TOPIC_AUTHORIZATION_FAILED, "Not authorized to create topic(s)"), topics); + } + private CreateTopicsResponse createTopicResponse(ApiError error, NewTopic... topics) { if (error == null) error = new ApiError(Errors.NONE, ""); Map topicResults = new HashMap<>(); diff --git a/connect/runtime/src/test/java/org/apache/kafka/connect/util/clusters/EmbeddedConnectCluster.java b/connect/runtime/src/test/java/org/apache/kafka/connect/util/clusters/EmbeddedConnectCluster.java new file mode 100644 index 0000000000000..a944d811a6f76 --- /dev/null +++ b/connect/runtime/src/test/java/org/apache/kafka/connect/util/clusters/EmbeddedConnectCluster.java @@ -0,0 +1,304 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.kafka.connect.util.clusters; + +import com.fasterxml.jackson.databind.ObjectMapper; +import org.apache.kafka.connect.cli.ConnectDistributed; +import org.apache.kafka.connect.errors.ConnectException; +import org.apache.kafka.connect.runtime.Connect; +import org.apache.kafka.connect.runtime.rest.entities.ConnectorStateInfo; +import org.apache.kafka.connect.runtime.rest.errors.ConnectRestException; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import javax.servlet.http.HttpServletResponse; +import javax.ws.rs.core.Response; +import java.io.IOException; +import java.io.InputStream; +import java.io.OutputStreamWriter; +import java.net.HttpURLConnection; +import java.net.URL; +import java.util.HashMap; +import java.util.Map; +import java.util.Properties; +import java.util.UUID; + +import static org.apache.kafka.clients.consumer.ConsumerConfig.GROUP_ID_CONFIG; +import static org.apache.kafka.connect.runtime.ConnectorConfig.KEY_CONVERTER_CLASS_CONFIG; +import static org.apache.kafka.connect.runtime.ConnectorConfig.VALUE_CONVERTER_CLASS_CONFIG; +import static org.apache.kafka.connect.runtime.WorkerConfig.BOOTSTRAP_SERVERS_CONFIG; +import static org.apache.kafka.connect.runtime.WorkerConfig.REST_HOST_NAME_CONFIG; +import static org.apache.kafka.connect.runtime.WorkerConfig.REST_PORT_CONFIG; +import static org.apache.kafka.connect.runtime.distributed.DistributedConfig.CONFIG_STORAGE_REPLICATION_FACTOR_CONFIG; +import static org.apache.kafka.connect.runtime.distributed.DistributedConfig.CONFIG_TOPIC_CONFIG; +import static org.apache.kafka.connect.runtime.distributed.DistributedConfig.OFFSET_STORAGE_REPLICATION_FACTOR_CONFIG; +import static org.apache.kafka.connect.runtime.distributed.DistributedConfig.OFFSET_STORAGE_TOPIC_CONFIG; +import static org.apache.kafka.connect.runtime.distributed.DistributedConfig.STATUS_STORAGE_REPLICATION_FACTOR_CONFIG; +import static org.apache.kafka.connect.runtime.distributed.DistributedConfig.STATUS_STORAGE_TOPIC_CONFIG; + +/** + * Start an embedded connect worker. Internally, this class will spin up a Kafka and Zk cluster, setup any tmp + * directories and clean up them on them. + */ +public class EmbeddedConnectCluster { + + private static final Logger log = LoggerFactory.getLogger(EmbeddedConnectCluster.class); + + private static final int DEFAULT_NUM_BROKERS = 1; + private static final int DEFAULT_NUM_WORKERS = 1; + private static final Properties DEFAULT_BROKER_CONFIG = new Properties(); + private static final String REST_HOST_NAME = "localhost"; + + private final Connect[] connectCluster; + private final EmbeddedKafkaCluster kafkaCluster; + private final Map workerProps; + private final String connectClusterName; + private final int numBrokers; + + private EmbeddedConnectCluster(String name, Map workerProps, int numWorkers, int numBrokers, Properties brokerProps) { + this.workerProps = workerProps; + this.connectClusterName = name; + this.numBrokers = numBrokers; + this.kafkaCluster = new EmbeddedKafkaCluster(numBrokers, brokerProps); + this.connectCluster = new Connect[numWorkers]; + } + + /** + * Start the connect cluster and the embedded Kafka and Zookeeper cluster. + */ + public void start() throws IOException { + kafkaCluster.before(); + startConnect(); + } + + /** + * Stop the connect cluster and the embedded Kafka and Zookeeper cluster. + * Clean up any temp directories created locally. + */ + public void stop() { + for (Connect worker : this.connectCluster) { + try { + worker.stop(); + } catch (Exception e) { + log.error("Could not stop connect", e); + throw new RuntimeException("Could not stop worker", e); + } + } + + try { + kafkaCluster.after(); + } catch (Exception e) { + log.error("Could not stop kafka", e); + throw new RuntimeException("Could not stop brokers", e); + } + } + + @SuppressWarnings("deprecation") + public void startConnect() { + log.info("Starting Connect cluster with {} workers. clusterName {}", connectCluster.length, connectClusterName); + + workerProps.put(BOOTSTRAP_SERVERS_CONFIG, kafka().bootstrapServers()); + workerProps.put(REST_HOST_NAME_CONFIG, REST_HOST_NAME); + workerProps.put(REST_PORT_CONFIG, "0"); // use a random available port + + String internalTopicsReplFactor = String.valueOf(numBrokers); + putIfAbsent(workerProps, GROUP_ID_CONFIG, "connect-integration-test-" + connectClusterName); + putIfAbsent(workerProps, OFFSET_STORAGE_TOPIC_CONFIG, "connect-offset-topic-" + connectClusterName); + putIfAbsent(workerProps, OFFSET_STORAGE_REPLICATION_FACTOR_CONFIG, internalTopicsReplFactor); + putIfAbsent(workerProps, CONFIG_TOPIC_CONFIG, "connect-config-topic-" + connectClusterName); + putIfAbsent(workerProps, CONFIG_STORAGE_REPLICATION_FACTOR_CONFIG, internalTopicsReplFactor); + putIfAbsent(workerProps, STATUS_STORAGE_TOPIC_CONFIG, "connect-storage-topic-" + connectClusterName); + putIfAbsent(workerProps, STATUS_STORAGE_REPLICATION_FACTOR_CONFIG, internalTopicsReplFactor); + putIfAbsent(workerProps, KEY_CONVERTER_CLASS_CONFIG, "org.apache.kafka.connect.storage.StringConverter"); + putIfAbsent(workerProps, VALUE_CONVERTER_CLASS_CONFIG, "org.apache.kafka.connect.storage.StringConverter"); + + for (int i = 0; i < connectCluster.length; i++) { + connectCluster[i] = new ConnectDistributed().startConnect(workerProps); + } + } + + /** + * Configure a connector. If the connector does not already exist, a new one will be created and + * the given configuration will be applied to it. + * + * @param connName the name of the connector + * @param connConfig the intended configuration + * @throws IOException if call to the REST api fails. + * @throws ConnectRestException if REST api returns error status + */ + public void configureConnector(String connName, Map connConfig) throws IOException { + String url = endpointForResource(String.format("connectors/%s/config", connName)); + ObjectMapper mapper = new ObjectMapper(); + int status; + try { + String content = mapper.writeValueAsString(connConfig); + status = executePut(url, content); + } catch (IOException e) { + log.error("Could not execute PUT request to " + url, e); + throw e; + } + if (status >= HttpServletResponse.SC_BAD_REQUEST) { + throw new ConnectRestException(status, "Could not execute PUT request"); + } + } + + /** + * Delete an existing connector. + * + * @param connName name of the connector to be deleted + * @throws IOException if call to the REST api fails. + */ + public void deleteConnector(String connName) throws IOException { + String url = endpointForResource(String.format("connectors/%s", connName)); + int status = executeDelete(url); + if (status >= HttpServletResponse.SC_BAD_REQUEST) { + throw new ConnectRestException(status, "Could not execute DELETE request."); + } + } + + /** + * Get the status for a connector running in this cluster. + * + * @param connectorName name of the connector + * @return an instance of {@link ConnectorStateInfo} populated with state informaton of the connector and it's tasks. + * @throws ConnectRestException if the HTTP request to the REST API failed with a valid status code. + * @throws ConnectException for any other error. + */ + public ConnectorStateInfo connectorStatus(String connectorName) { + ObjectMapper mapper = new ObjectMapper(); + String url = endpointForResource(String.format("connectors/%s/status", connectorName)); + try { + return mapper.readerFor(ConnectorStateInfo.class).readValue(executeGet(url)); + } catch (IOException e) { + log.error("Could not read connector state", e); + throw new ConnectException("Could not read connector state", e); + } + } + + public String endpointForResource(String resource) { + String url = String.valueOf(connectCluster[0].restUrl()); + return url + resource; + } + + private static void putIfAbsent(Map props, String propertyKey, String propertyValue) { + if (!props.containsKey(propertyKey)) { + props.put(propertyKey, propertyValue); + } + } + + public EmbeddedKafkaCluster kafka() { + return kafkaCluster; + } + + public int executePut(String url, String body) throws IOException { + log.debug("Executing PUT request to URL={}. Payload={}", url, body); + HttpURLConnection httpCon = (HttpURLConnection) new URL(url).openConnection(); + httpCon.setDoOutput(true); + httpCon.setRequestProperty("Content-Type", "application/json"); + httpCon.setRequestMethod("PUT"); + try (OutputStreamWriter out = new OutputStreamWriter(httpCon.getOutputStream())) { + out.write(body); + } + try (InputStream is = httpCon.getInputStream()) { + int c; + StringBuilder response = new StringBuilder(); + while ((c = is.read()) != -1) { + response.append((char) c); + } + log.info("Put response for URL={} is {}", url, response); + } + return httpCon.getResponseCode(); + } + + /** + * Execute a GET request on the given URL. + * + * @param url the HTTP endpoint + * @return response body encoded as a String + * @throws ConnectRestException if the HTTP request fails with a valid status code + * @throws IOException for any other I/O error. + */ + public String executeGet(String url) throws IOException { + log.debug("Executing GET request to URL={}.", url); + HttpURLConnection httpCon = (HttpURLConnection) new URL(url).openConnection(); + httpCon.setDoOutput(true); + httpCon.setRequestMethod("GET"); + try (InputStream is = httpCon.getInputStream()) { + int c; + StringBuilder response = new StringBuilder(); + while ((c = is.read()) != -1) { + response.append((char) c); + } + log.debug("Get response for URL={} is {}", url, response); + return response.toString(); + } catch (IOException e) { + Response.Status status = Response.Status.fromStatusCode(httpCon.getResponseCode()); + if (status != null) { + throw new ConnectRestException(status, "Invalid endpoint: " + url, e); + } + // invalid response code, re-throw the IOException. + throw e; + } + } + + public int executeDelete(String url) throws IOException { + log.debug("Executing DELETE request to URL={}", url); + HttpURLConnection httpCon = (HttpURLConnection) new URL(url).openConnection(); + httpCon.setDoOutput(true); + httpCon.setRequestMethod("DELETE"); + httpCon.connect(); + return httpCon.getResponseCode(); + } + + public static class Builder { + private String name = UUID.randomUUID().toString(); + private Map workerProps = new HashMap<>(); + private int numWorkers = DEFAULT_NUM_WORKERS; + private int numBrokers = DEFAULT_NUM_BROKERS; + private Properties brokerProps = DEFAULT_BROKER_CONFIG; + + public Builder name(String name) { + this.name = name; + return this; + } + + public Builder workerProps(Map workerProps) { + this.workerProps = workerProps; + return this; + } + + public Builder numWorkers(int numWorkers) { + this.numWorkers = numWorkers; + return this; + } + + public Builder numBrokers(int numBrokers) { + this.numBrokers = numBrokers; + return this; + } + + public Builder brokerProps(Properties brokerProps) { + this.brokerProps = brokerProps; + return this; + } + + public EmbeddedConnectCluster build() { + return new EmbeddedConnectCluster(name, workerProps, numWorkers, numBrokers, brokerProps); + } + } + +} diff --git a/connect/runtime/src/test/java/org/apache/kafka/connect/util/clusters/EmbeddedKafkaCluster.java b/connect/runtime/src/test/java/org/apache/kafka/connect/util/clusters/EmbeddedKafkaCluster.java new file mode 100644 index 0000000000000..44643950b4ba6 --- /dev/null +++ b/connect/runtime/src/test/java/org/apache/kafka/connect/util/clusters/EmbeddedKafkaCluster.java @@ -0,0 +1,339 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.kafka.connect.util.clusters; + +import kafka.server.KafkaConfig; +import kafka.server.KafkaConfig$; +import kafka.server.KafkaServer; +import kafka.utils.CoreUtils; +import kafka.utils.TestUtils; +import kafka.zk.EmbeddedZookeeper; +import org.apache.kafka.clients.CommonClientConfigs; +import org.apache.kafka.clients.admin.AdminClient; +import org.apache.kafka.clients.admin.AdminClientConfig; +import org.apache.kafka.clients.admin.NewTopic; +import org.apache.kafka.clients.consumer.ConsumerRecord; +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.ProducerConfig; +import org.apache.kafka.clients.producer.ProducerRecord; +import org.apache.kafka.common.KafkaException; +import org.apache.kafka.common.TopicPartition; +import org.apache.kafka.common.config.SslConfigs; +import org.apache.kafka.common.config.types.Password; +import org.apache.kafka.common.errors.InvalidReplicationFactorException; +import org.apache.kafka.common.network.ListenerName; +import org.apache.kafka.common.utils.MockTime; +import org.apache.kafka.common.utils.Time; +import org.apache.kafka.connect.errors.ConnectException; +import org.junit.rules.ExternalResource; +import org.junit.rules.TemporaryFolder; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.IOException; +import java.time.Duration; +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 java.util.Properties; +import java.util.UUID; +import java.util.concurrent.ExecutionException; +import java.util.concurrent.TimeUnit; +import java.util.stream.Collectors; + +import static org.apache.kafka.clients.consumer.ConsumerConfig.AUTO_OFFSET_RESET_CONFIG; +import static org.apache.kafka.clients.consumer.ConsumerConfig.BOOTSTRAP_SERVERS_CONFIG; +import static org.apache.kafka.clients.consumer.ConsumerConfig.ENABLE_AUTO_COMMIT_CONFIG; +import static org.apache.kafka.clients.consumer.ConsumerConfig.GROUP_ID_CONFIG; +import static org.apache.kafka.clients.consumer.ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG; +import static org.apache.kafka.clients.consumer.ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG; + +/** + * Setup an embedded Kafka cluster with specified number of brokers and specified broker properties. To be used for + * integration tests. + */ +public class EmbeddedKafkaCluster extends ExternalResource { + + private static final Logger log = LoggerFactory.getLogger(EmbeddedKafkaCluster.class); + + private static final long DEFAULT_PRODUCE_SEND_DURATION_MS = TimeUnit.SECONDS.toMillis(120); + + // Kafka Config + private final KafkaServer[] brokers; + private final Properties brokerConfig; + private final Time time = new MockTime(); + + private EmbeddedZookeeper zookeeper = null; + private ListenerName listenerName = new ListenerName("PLAINTEXT"); + private KafkaProducer producer; + + public EmbeddedKafkaCluster(final int numBrokers, + final Properties brokerConfig) { + brokers = new KafkaServer[numBrokers]; + this.brokerConfig = brokerConfig; + } + + @Override + protected void before() throws IOException { + start(); + } + + @Override + protected void after() { + stop(); + } + + private void start() throws IOException { + zookeeper = new EmbeddedZookeeper(); + + brokerConfig.put(KafkaConfig$.MODULE$.ZkConnectProp(), zKConnectString()); + brokerConfig.put(KafkaConfig$.MODULE$.PortProp(), 0); // pick a random port + + putIfAbsent(brokerConfig, KafkaConfig$.MODULE$.HostNameProp(), "localhost"); + putIfAbsent(brokerConfig, KafkaConfig$.MODULE$.DeleteTopicEnableProp(), true); + putIfAbsent(brokerConfig, KafkaConfig$.MODULE$.GroupInitialRebalanceDelayMsProp(), 0); + putIfAbsent(brokerConfig, KafkaConfig$.MODULE$.OffsetsTopicReplicationFactorProp(), (short) brokers.length); + putIfAbsent(brokerConfig, KafkaConfig$.MODULE$.AutoCreateTopicsEnableProp(), false); + + Object listenerConfig = brokerConfig.get(KafkaConfig$.MODULE$.InterBrokerListenerNameProp()); + if (listenerConfig != null) { + listenerName = new ListenerName(listenerConfig.toString()); + } + + for (int i = 0; i < brokers.length; i++) { + brokerConfig.put(KafkaConfig$.MODULE$.BrokerIdProp(), i); + brokerConfig.put(KafkaConfig$.MODULE$.LogDirProp(), createLogDir()); + brokers[i] = TestUtils.createServer(new KafkaConfig(brokerConfig, true), time); + } + + Map producerProps = new HashMap<>(); + producerProps.put(ProducerConfig.BOOTSTRAP_SERVERS_CONFIG, bootstrapServers()); + 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"); + producer = new KafkaProducer<>(producerProps); + } + + private void stop() { + + try { + producer.close(); + } catch (Exception e) { + log.error("Could not shutdown producer ", e); + throw new RuntimeException("Could not shutdown producer", e); + } + + for (KafkaServer broker : brokers) { + try { + broker.shutdown(); + } catch (Throwable t) { + String msg = String.format("Could not shutdown broker at %s", address(broker)); + log.error(msg, t); + throw new RuntimeException(msg, t); + } + } + + for (KafkaServer broker : brokers) { + try { + log.info("Cleaning up kafka log dirs at {}", broker.config().logDirs()); + CoreUtils.delete(broker.config().logDirs()); + } catch (Throwable t) { + String msg = String.format("Could not clean up log dirs for broker at %s", address(broker)); + log.error(msg, t); + throw new RuntimeException(msg, t); + } + } + + try { + zookeeper.shutdown(); + } catch (Throwable t) { + String msg = String.format("Could not shutdown zookeeper at %s", zKConnectString()); + log.error(msg, t); + throw new RuntimeException(msg, t); + } + } + + private static void putIfAbsent(final Properties props, final String propertyKey, final Object propertyValue) { + if (!props.containsKey(propertyKey)) { + props.put(propertyKey, propertyValue); + } + } + + private String createLogDir() throws IOException { + TemporaryFolder tmpFolder = new TemporaryFolder(); + tmpFolder.create(); + return tmpFolder.newFolder().getAbsolutePath(); + } + + public String bootstrapServers() { + return Arrays.stream(brokers) + .map(this::address) + .collect(Collectors.joining(",")); + } + + public String address(KafkaServer server) { + return server.config().hostName() + ":" + server.boundPort(listenerName); + } + + public String zKConnectString() { + return "127.0.0.1:" + zookeeper.port(); + } + + /** + * Create a Kafka topic with 1 partition and a replication factor of 1. + * + * @param topic The name of the topic. + */ + public void createTopic(String topic) { + createTopic(topic, 1); + } + + /** + * Create a Kafka topic with given partition and a replication factor of 1. + * + * @param topic The name of the topic. + */ + public void createTopic(String topic, int partitions) { + createTopic(topic, partitions, 1, new HashMap<>()); + } + + /** + * Create a Kafka topic with the given parameters. + * + * @param topic The name of the topic. + * @param partitions The number of partitions for this topic. + * @param replication The replication factor for (partitions of) this topic. + * @param topicConfig Additional topic-level configuration settings. + */ + public void createTopic(String topic, int partitions, int replication, Map topicConfig) { + if (replication > brokers.length) { + throw new InvalidReplicationFactorException("Insufficient brokers (" + + brokers.length + ") for desired replication (" + replication + ")"); + } + + log.debug("Creating topic { name: {}, partitions: {}, replication: {}, config: {} }", + topic, partitions, replication, topicConfig); + final NewTopic newTopic = new NewTopic(topic, partitions, (short) replication); + newTopic.configs(topicConfig); + + try (final AdminClient adminClient = createAdminClient()) { + adminClient.createTopics(Collections.singletonList(newTopic)).all().get(); + } catch (final InterruptedException | ExecutionException e) { + throw new RuntimeException(e); + } + } + + public void produce(String topic, String value) { + produce(topic, null, null, value); + } + + public void produce(String topic, String key, String value) { + produce(topic, null, key, value); + } + + public void produce(String topic, Integer partition, String key, String value) { + ProducerRecord msg = new ProducerRecord<>(topic, partition, key == null ? null : key.getBytes(), value == null ? null : value.getBytes()); + try { + producer.send(msg).get(DEFAULT_PRODUCE_SEND_DURATION_MS, TimeUnit.MILLISECONDS); + } catch (Exception e) { + throw new KafkaException("Could not produce message: " + msg, e); + } + } + + public AdminClient createAdminClient() { + final Properties adminClientConfig = new Properties(); + adminClientConfig.put(AdminClientConfig.BOOTSTRAP_SERVERS_CONFIG, bootstrapServers()); + final Object listeners = brokerConfig.get(KafkaConfig$.MODULE$.ListenersProp()); + if (listeners != null && listeners.toString().contains("SSL")) { + adminClientConfig.put(SslConfigs.SSL_TRUSTSTORE_LOCATION_CONFIG, brokerConfig.get(SslConfigs.SSL_TRUSTSTORE_LOCATION_CONFIG)); + adminClientConfig.put(SslConfigs.SSL_TRUSTSTORE_PASSWORD_CONFIG, ((Password) brokerConfig.get(SslConfigs.SSL_TRUSTSTORE_PASSWORD_CONFIG)).value()); + adminClientConfig.put(CommonClientConfigs.SECURITY_PROTOCOL_CONFIG, "SSL"); + } + return AdminClient.create(adminClientConfig); + } + + /** + * Consume at least n records in a given duration or throw an exception. + * + * @param n the number of expected records in this topic. + * @param maxDuration the max duration to wait for these records (in milliseconds). + * @param topics the topics to subscribe and consume records from. + * @return a {@link ConsumerRecords} collection containing at least n records. + */ + public ConsumerRecords consume(int n, long maxDuration, String... topics) { + Map>> records = new HashMap<>(); + int consumedRecords = 0; + try (KafkaConsumer consumer = createConsumerAndSubscribeTo(Collections.emptyMap(), topics)) { + final long startMillis = System.currentTimeMillis(); + long allowedDuration = maxDuration; + while (allowedDuration > 0) { + log.debug("Consuming from {} for {} millis.", Arrays.toString(topics), allowedDuration); + ConsumerRecords rec = consumer.poll(Duration.ofMillis(allowedDuration)); + if (rec.isEmpty()) { + allowedDuration = maxDuration - (System.currentTimeMillis() - startMillis); + continue; + } + for (TopicPartition partition: rec.partitions()) { + final List> r = rec.records(partition); + records.computeIfAbsent(partition, t -> new ArrayList<>()).addAll(r); + consumedRecords += r.size(); + } + if (consumedRecords >= n) { + return new ConsumerRecords<>(records); + } + allowedDuration = maxDuration - (System.currentTimeMillis() - startMillis); + } + } + + throw new RuntimeException("Could not find enough records. found " + consumedRecords + ", expected " + n); + } + + public KafkaConsumer createConsumer(Map consumerProps) { + Map props = new HashMap<>(consumerProps); + + putIfAbsent(props, GROUP_ID_CONFIG, UUID.randomUUID().toString()); + putIfAbsent(props, BOOTSTRAP_SERVERS_CONFIG, bootstrapServers()); + putIfAbsent(props, ENABLE_AUTO_COMMIT_CONFIG, "false"); + putIfAbsent(props, AUTO_OFFSET_RESET_CONFIG, "earliest"); + putIfAbsent(props, KEY_DESERIALIZER_CLASS_CONFIG, "org.apache.kafka.common.serialization.ByteArrayDeserializer"); + putIfAbsent(props, VALUE_DESERIALIZER_CLASS_CONFIG, "org.apache.kafka.common.serialization.ByteArrayDeserializer"); + + KafkaConsumer consumer; + try { + consumer = new KafkaConsumer<>(props); + } catch (Throwable t) { + throw new ConnectException("Failed to create consumer", t); + } + return consumer; + } + + public KafkaConsumer createConsumerAndSubscribeTo(Map consumerProps, String... topics) { + KafkaConsumer consumer = createConsumer(consumerProps); + consumer.subscribe(Arrays.asList(topics)); + return consumer; + } + + private static void putIfAbsent(final Map props, final String propertyKey, final Object propertyValue) { + if (!props.containsKey(propertyKey)) { + props.put(propertyKey, propertyValue); + } + } +} diff --git a/connect/runtime/src/test/resources/log4j.properties b/connect/runtime/src/test/resources/log4j.properties index d5e90fe788f76..a396aeef657d9 100644 --- a/connect/runtime/src/test/resources/log4j.properties +++ b/connect/runtime/src/test/resources/log4j.properties @@ -14,10 +14,14 @@ # See the License for the specific language governing permissions and # limitations under the License. ## -log4j.rootLogger=OFF, stdout +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.appender.stdout.layout.ConversionPattern=[%d] (%t) %p %m (%c:%L)%n -log4j.logger.org.apache.kafka=ERROR +log4j.logger.org.reflections=ERROR +log4j.logger.kafka=WARN +log4j.logger.org.apache.kafka.connect=DEBUG +log4j.logger.org.apache.kafka.connect.runtime.distributed=DEBUG +log4j.logger.org.apache.kafka.connect.integration=DEBUG diff --git a/connect/runtime/src/test/resources/test-plugins/aliased-static-field/test/plugins/AliasedStaticField.java b/connect/runtime/src/test/resources/test-plugins/aliased-static-field/test/plugins/AliasedStaticField.java new file mode 100644 index 0000000000000..d865f4e91ba86 --- /dev/null +++ b/connect/runtime/src/test/resources/test-plugins/aliased-static-field/test/plugins/AliasedStaticField.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 test.plugins; + +import java.util.Map; +import java.util.HashMap; +import org.apache.kafka.connect.data.Schema; +import org.apache.kafka.connect.data.SchemaAndValue; +import org.apache.kafka.connect.storage.Converter; +import org.apache.kafka.connect.runtime.isolation.SamplingTestPlugin; + +/** + * Samples data about its initialization environment for later analysis + * Samples are shared between instances of the same class in a static variable + */ +public class AliasedStaticField extends SamplingTestPlugin implements Converter { + + private static final Map SAMPLES; + private static final ClassLoader STATIC_CLASS_LOADER; + private final ClassLoader classloader; + + static { + SAMPLES = new HashMap<>(); + STATIC_CLASS_LOADER = Thread.currentThread().getContextClassLoader(); + } + + { + classloader = Thread.currentThread().getContextClassLoader(); + } + + @Override + public void configure(final Map configs, final boolean isKey) { + + } + + @Override + public byte[] fromConnectData(final String topic, final Schema schema, final Object value) { + return new byte[0]; + } + + @Override + public SchemaAndValue toConnectData(final String topic, final byte[] value) { + return null; + } + + @Override + public ClassLoader staticClassloader() { + return STATIC_CLASS_LOADER; + } + + @Override + public ClassLoader classloader() { + return classloader; + } + + @Override + public Map otherSamples() { + return SAMPLES; + } +} diff --git a/clients/src/main/java/org/apache/kafka/common/requests/Resource.java b/connect/runtime/src/test/resources/test-plugins/always-throw-exception/test/plugins/AlwaysThrowException.java similarity index 50% rename from clients/src/main/java/org/apache/kafka/common/requests/Resource.java rename to connect/runtime/src/test/resources/test-plugins/always-throw-exception/test/plugins/AlwaysThrowException.java index bd814661ae344..858f3ed5eacbe 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/Resource.java +++ b/connect/runtime/src/test/resources/test-plugins/always-throw-exception/test/plugins/AlwaysThrowException.java @@ -15,46 +15,39 @@ * limitations under the License. */ -package org.apache.kafka.common.requests; +package test.plugins; -public final class Resource { - private final ResourceType type; - private final String name; +import java.util.Map; +import org.apache.kafka.connect.data.Schema; +import org.apache.kafka.connect.data.SchemaAndValue; +import org.apache.kafka.connect.runtime.isolation.SamplingTestPlugin; +import org.apache.kafka.connect.storage.Converter; - public Resource(ResourceType type, String name) { - this.type = type; - this.name = name; - } +/** + * Unconditionally throw an exception during static initialization. + */ +public class AlwaysThrowException implements Converter { - public ResourceType type() { - return type; + static { + setup(); } - public String name() { - return name; + public static void setup() { + throw new RuntimeException("I always throw an exception"); } @Override - public boolean equals(Object o) { - if (this == o) - return true; - if (o == null || getClass() != o.getClass()) - return false; - - Resource resource = (Resource) o; + public void configure(final Map configs, final boolean isKey) { - return type == resource.type && name.equals(resource.name); } @Override - public int hashCode() { - int result = type.hashCode(); - result = 31 * result + name.hashCode(); - return result; + public byte[] fromConnectData(final String topic, final Schema schema, final Object value) { + return new byte[0]; } @Override - public String toString() { - return "Resource(type=" + type + ", name='" + name + "')"; + public SchemaAndValue toConnectData(final String topic, final byte[] value) { + return null; } } diff --git a/connect/runtime/src/test/resources/test-plugins/sampling-config-provider/META-INF/services/org.apache.kafka.common.config.provider.ConfigProvider b/connect/runtime/src/test/resources/test-plugins/sampling-config-provider/META-INF/services/org.apache.kafka.common.config.provider.ConfigProvider new file mode 100644 index 0000000000000..62d8df254bbc3 --- /dev/null +++ b/connect/runtime/src/test/resources/test-plugins/sampling-config-provider/META-INF/services/org.apache.kafka.common.config.provider.ConfigProvider @@ -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. + +test.plugins.SamplingConfigProvider diff --git a/connect/runtime/src/test/resources/test-plugins/sampling-config-provider/test/plugins/SamplingConfigProvider.java b/connect/runtime/src/test/resources/test-plugins/sampling-config-provider/test/plugins/SamplingConfigProvider.java new file mode 100644 index 0000000000000..df8285eba9a1a --- /dev/null +++ b/connect/runtime/src/test/resources/test-plugins/sampling-config-provider/test/plugins/SamplingConfigProvider.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 test.plugins; + +import java.util.Set; +import java.util.Map; +import java.util.HashMap; +import org.apache.kafka.common.config.provider.ConfigProvider; +import org.apache.kafka.common.config.ConfigData; +import org.apache.kafka.common.config.ConfigChangeCallback; +import org.apache.kafka.connect.data.Schema; +import org.apache.kafka.connect.data.SchemaAndValue; +import org.apache.kafka.connect.storage.Converter; +import org.apache.kafka.connect.runtime.isolation.SamplingTestPlugin; +import org.apache.kafka.connect.storage.HeaderConverter; + +/** + * Samples data about its initialization environment for later analysis + */ +public class SamplingConfigProvider extends SamplingTestPlugin implements ConfigProvider { + + private static final ClassLoader STATIC_CLASS_LOADER; + private final ClassLoader classloader; + private Map samples; + + static { + STATIC_CLASS_LOADER = Thread.currentThread().getContextClassLoader(); + } + + { + samples = new HashMap<>(); + classloader = Thread.currentThread().getContextClassLoader(); + } + + @Override + public ConfigData get(String path) { + logMethodCall(samples); + return null; + } + + @Override + public ConfigData get(String path, Set keys) { + logMethodCall(samples); + return null; + } + + @Override + public void subscribe(String path, Set keys, ConfigChangeCallback callback) { + logMethodCall(samples); + } + + @Override + public void unsubscribe(String path, Set keys, ConfigChangeCallback callback) { + logMethodCall(samples); + } + + @Override + public void unsubscribeAll() { + logMethodCall(samples); + } + + @Override + public void configure(final Map configs) { + logMethodCall(samples); + } + + @Override + public void close() { + logMethodCall(samples); + } + + @Override + public ClassLoader staticClassloader() { + return STATIC_CLASS_LOADER; + } + + @Override + public ClassLoader classloader() { + return classloader; + } + + @Override + public Map otherSamples() { + return samples; + } +} diff --git a/connect/runtime/src/test/resources/test-plugins/sampling-configurable/test/plugins/SamplingConfigurable.java b/connect/runtime/src/test/resources/test-plugins/sampling-configurable/test/plugins/SamplingConfigurable.java new file mode 100644 index 0000000000000..a917f2f2ca1a8 --- /dev/null +++ b/connect/runtime/src/test/resources/test-plugins/sampling-configurable/test/plugins/SamplingConfigurable.java @@ -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 test.plugins; + +import java.util.Map; +import java.util.HashMap; +import org.apache.kafka.common.Configurable; +import org.apache.kafka.connect.data.Schema; +import org.apache.kafka.connect.data.SchemaAndValue; +import org.apache.kafka.connect.storage.Converter; +import org.apache.kafka.connect.runtime.isolation.SamplingTestPlugin; + +/** + * Samples data about its initialization environment for later analysis + */ +public class SamplingConfigurable extends SamplingTestPlugin implements Converter, Configurable { + + private static final ClassLoader STATIC_CLASS_LOADER; + private final ClassLoader classloader; + private Map samples; + + static { + STATIC_CLASS_LOADER = Thread.currentThread().getContextClassLoader(); + } + + { + samples = new HashMap<>(); + classloader = Thread.currentThread().getContextClassLoader(); + } + + @Override + public void configure(final Map configs) { + logMethodCall(samples); + } + + @Override + public void configure(final Map configs, final boolean isKey) { + } + + @Override + public byte[] fromConnectData(final String topic, final Schema schema, final Object value) { + return new byte[0]; + } + + @Override + public SchemaAndValue toConnectData(final String topic, final byte[] value) { + return null; + } + + @Override + public ClassLoader staticClassloader() { + return STATIC_CLASS_LOADER; + } + + @Override + public ClassLoader classloader() { + return classloader; + } + + @Override + public Map otherSamples() { + return samples; + } +} diff --git a/connect/runtime/src/test/resources/test-plugins/sampling-converter/test/plugins/SamplingConverter.java b/connect/runtime/src/test/resources/test-plugins/sampling-converter/test/plugins/SamplingConverter.java new file mode 100644 index 0000000000000..39109a1d4e573 --- /dev/null +++ b/connect/runtime/src/test/resources/test-plugins/sampling-converter/test/plugins/SamplingConverter.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 test.plugins; + +import java.util.Map; +import java.util.HashMap; +import org.apache.kafka.connect.data.Schema; +import org.apache.kafka.connect.data.SchemaAndValue; +import org.apache.kafka.connect.storage.Converter; +import org.apache.kafka.connect.runtime.isolation.SamplingTestPlugin; + +/** + * Samples data about its initialization environment for later analysis + */ +public class SamplingConverter extends SamplingTestPlugin implements Converter { + + private static final ClassLoader STATIC_CLASS_LOADER; + private final ClassLoader classloader; + private Map samples; + + static { + STATIC_CLASS_LOADER = Thread.currentThread().getContextClassLoader(); + } + + { + samples = new HashMap<>(); + classloader = Thread.currentThread().getContextClassLoader(); + } + + @Override + public void configure(final Map configs, final boolean isKey) { + logMethodCall(samples); + } + + @Override + public byte[] fromConnectData(final String topic, final Schema schema, final Object value) { + logMethodCall(samples); + return new byte[0]; + } + + @Override + public SchemaAndValue toConnectData(final String topic, final byte[] value) { + logMethodCall(samples); + return null; + } + + @Override + public ClassLoader staticClassloader() { + return STATIC_CLASS_LOADER; + } + + @Override + public ClassLoader classloader() { + return classloader; + } + + @Override + public Map otherSamples() { + return samples; + } +} diff --git a/connect/runtime/src/test/resources/test-plugins/sampling-header-converter/test/plugins/SamplingHeaderConverter.java b/connect/runtime/src/test/resources/test-plugins/sampling-header-converter/test/plugins/SamplingHeaderConverter.java new file mode 100644 index 0000000000000..11a1e28e7278c --- /dev/null +++ b/connect/runtime/src/test/resources/test-plugins/sampling-header-converter/test/plugins/SamplingHeaderConverter.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 test.plugins; + +import java.util.Map; +import java.util.HashMap; +import org.apache.kafka.common.config.ConfigDef; +import org.apache.kafka.connect.data.Schema; +import org.apache.kafka.connect.data.SchemaAndValue; +import org.apache.kafka.connect.storage.Converter; +import org.apache.kafka.connect.runtime.isolation.SamplingTestPlugin; +import org.apache.kafka.connect.storage.HeaderConverter; + +/** + * Samples data about its initialization environment for later analysis + */ +public class SamplingHeaderConverter extends SamplingTestPlugin implements HeaderConverter { + + private static final ClassLoader STATIC_CLASS_LOADER; + private final ClassLoader classloader; + private Map samples; + + static { + STATIC_CLASS_LOADER = Thread.currentThread().getContextClassLoader(); + } + + { + samples = new HashMap<>(); + classloader = Thread.currentThread().getContextClassLoader(); + } + + @Override + public SchemaAndValue toConnectHeader(String topic, String headerKey, byte[] value) { + logMethodCall(samples); + return null; + } + + @Override + public byte[] fromConnectHeader(String topic, String headerKey, Schema schema, Object value) { + logMethodCall(samples); + return new byte[0]; + } + + @Override + public ConfigDef config() { + logMethodCall(samples); + return null; + } + + @Override + public void configure(final Map configs) { + logMethodCall(samples); + } + + @Override + public void close() { + logMethodCall(samples); + } + + @Override + public ClassLoader staticClassloader() { + return STATIC_CLASS_LOADER; + } + + @Override + public ClassLoader classloader() { + return classloader; + } + + @Override + public Map otherSamples() { + return samples; + } +} diff --git a/connect/runtime/src/test/resources/test-plugins/service-loader/META-INF/services/test.plugins.ServiceLoadedClass b/connect/runtime/src/test/resources/test-plugins/service-loader/META-INF/services/test.plugins.ServiceLoadedClass new file mode 100644 index 0000000000000..b8db8656487d2 --- /dev/null +++ b/connect/runtime/src/test/resources/test-plugins/service-loader/META-INF/services/test.plugins.ServiceLoadedClass @@ -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. + +test.plugins.ServiceLoadedSubclass \ No newline at end of file diff --git a/clients/src/main/java/org/apache/kafka/common/errors/GroupLoadInProgressException.java b/connect/runtime/src/test/resources/test-plugins/service-loader/test/plugins/ServiceLoadedClass.java similarity index 50% rename from clients/src/main/java/org/apache/kafka/common/errors/GroupLoadInProgressException.java rename to connect/runtime/src/test/resources/test-plugins/service-loader/test/plugins/ServiceLoadedClass.java index 73daa5f1c3a55..98677ed43d65d 100644 --- a/clients/src/main/java/org/apache/kafka/common/errors/GroupLoadInProgressException.java +++ b/connect/runtime/src/test/resources/test-plugins/service-loader/test/plugins/ServiceLoadedClass.java @@ -14,33 +14,35 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.kafka.common.errors; + +package test.plugins; + +import org.apache.kafka.connect.runtime.isolation.SamplingTestPlugin; /** - * The broker returns this error code for any coordinator request if it is still loading the metadata (after a leader change - * for that offsets topic partition) for this group. - * - * @deprecated As of Kafka 0.11, this has been replaced by {@link CoordinatorLoadInProgressException} + * Superclass for service loaded classes */ -@Deprecated -public class GroupLoadInProgressException extends RetriableException { +public class ServiceLoadedClass extends SamplingTestPlugin { - private static final long serialVersionUID = 1L; + private static final ClassLoader STATIC_CLASS_LOADER; + private final ClassLoader classloader; - public GroupLoadInProgressException() { - super(); - } + static { + STATIC_CLASS_LOADER = Thread.currentThread().getContextClassLoader(); + } - public GroupLoadInProgressException(String message) { - super(message); - } + { + classloader = Thread.currentThread().getContextClassLoader(); + } - public GroupLoadInProgressException(String message, Throwable cause) { - super(message, cause); - } + @Override + public ClassLoader staticClassloader() { + return STATIC_CLASS_LOADER; + } - public GroupLoadInProgressException(Throwable cause) { - super(cause); - } + @Override + public ClassLoader classloader() { + return classloader; + } } diff --git a/clients/src/main/java/org/apache/kafka/common/requests/ResourceType.java b/connect/runtime/src/test/resources/test-plugins/service-loader/test/plugins/ServiceLoadedSubclass.java similarity index 57% rename from clients/src/main/java/org/apache/kafka/common/requests/ResourceType.java rename to connect/runtime/src/test/resources/test-plugins/service-loader/test/plugins/ServiceLoadedSubclass.java index 2c117727f93d6..cfc6b6f9cfc94 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/ResourceType.java +++ b/connect/runtime/src/test/resources/test-plugins/service-loader/test/plugins/ServiceLoadedSubclass.java @@ -15,28 +15,32 @@ * limitations under the License. */ -package org.apache.kafka.common.requests; +package test.plugins; -public enum ResourceType { - UNKNOWN((byte) 0), ANY((byte) 1), TOPIC((byte) 2), GROUP((byte) 3), BROKER((byte) 4); +/** + * Instance of a service loaded class + */ +public class ServiceLoadedSubclass extends ServiceLoadedClass { + + private static final ClassLoader STATIC_CLASS_LOADER; + private final ClassLoader classloader; - private static final ResourceType[] VALUES = values(); + static { + STATIC_CLASS_LOADER = Thread.currentThread().getContextClassLoader(); + } - private final byte id; + { + classloader = Thread.currentThread().getContextClassLoader(); + } - ResourceType(byte id) { - this.id = id; - } + @Override + public ClassLoader staticClassloader() { + return STATIC_CLASS_LOADER; + } - public byte id() { - return id; - } + @Override + public ClassLoader classloader() { + return classloader; + } - public static ResourceType forId(byte id) { - if (id < 0) - throw new IllegalArgumentException("id should be positive, id: " + id); - if (id >= VALUES.length) - return UNKNOWN; - return VALUES[id]; - } } diff --git a/connect/runtime/src/test/resources/test-plugins/service-loader/test/plugins/ServiceLoaderPlugin.java b/connect/runtime/src/test/resources/test-plugins/service-loader/test/plugins/ServiceLoaderPlugin.java new file mode 100644 index 0000000000000..e6371baf56aad --- /dev/null +++ b/connect/runtime/src/test/resources/test-plugins/service-loader/test/plugins/ServiceLoaderPlugin.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 test.plugins; + +import java.util.Map; +import java.util.HashMap; +import java.util.ServiceLoader; +import java.util.Iterator; +import org.apache.kafka.connect.data.Schema; +import org.apache.kafka.connect.data.SchemaAndValue; +import org.apache.kafka.connect.storage.Converter; +import org.apache.kafka.connect.runtime.isolation.SamplingTestPlugin; + +/** + * Samples data about its initialization environment for later analysis + */ +public class ServiceLoaderPlugin extends SamplingTestPlugin implements Converter { + + private static final ClassLoader STATIC_CLASS_LOADER; + private static final Map SAMPLES; + private final ClassLoader classloader; + + static { + STATIC_CLASS_LOADER = Thread.currentThread().getContextClassLoader(); + SAMPLES = new HashMap<>(); + Iterator it = ServiceLoader.load(ServiceLoadedClass.class).iterator(); + while (it.hasNext()) { + ServiceLoadedClass loaded = it.next(); + SAMPLES.put(loaded.getClass().getSimpleName() + ".static", loaded); + } + } + + { + classloader = Thread.currentThread().getContextClassLoader(); + Iterator it = ServiceLoader.load(ServiceLoadedClass.class).iterator(); + while (it.hasNext()) { + ServiceLoadedClass loaded = it.next(); + SAMPLES.put(loaded.getClass().getSimpleName() + ".dynamic", loaded); + } + } + + @Override + public void configure(final Map configs, final boolean isKey) { + } + + @Override + public byte[] fromConnectData(final String topic, final Schema schema, final Object value) { + return new byte[0]; + } + + @Override + public SchemaAndValue toConnectData(final String topic, final byte[] value) { + return null; + } + + @Override + public ClassLoader staticClassloader() { + return STATIC_CLASS_LOADER; + } + + @Override + public ClassLoader classloader() { + return classloader; + } + + @Override + public Map otherSamples() { + return SAMPLES; + } +} diff --git a/connect/transforms/src/main/java/org/apache/kafka/connect/transforms/Cast.java b/connect/transforms/src/main/java/org/apache/kafka/connect/transforms/Cast.java index d94f8f648b3fe..9dcec15eace94 100644 --- a/connect/transforms/src/main/java/org/apache/kafka/connect/transforms/Cast.java +++ b/connect/transforms/src/main/java/org/apache/kafka/connect/transforms/Cast.java @@ -24,13 +24,20 @@ import org.apache.kafka.common.config.ConfigException; import org.apache.kafka.connect.connector.ConnectRecord; import org.apache.kafka.connect.data.ConnectSchema; +import org.apache.kafka.connect.data.Date; import org.apache.kafka.connect.data.Field; import org.apache.kafka.connect.data.Schema; +import org.apache.kafka.connect.data.Schema.Type; import org.apache.kafka.connect.data.SchemaBuilder; import org.apache.kafka.connect.data.Struct; +import org.apache.kafka.connect.data.Time; +import org.apache.kafka.connect.data.Timestamp; +import org.apache.kafka.connect.data.Values; import org.apache.kafka.connect.errors.DataException; import org.apache.kafka.connect.transforms.util.SchemaUtil; import org.apache.kafka.connect.transforms.util.SimpleConfig; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; import java.util.EnumSet; import java.util.HashMap; @@ -43,6 +50,7 @@ import static org.apache.kafka.connect.transforms.util.Requirements.requireStruct; public abstract class Cast> implements Transformation { + private static final Logger log = LoggerFactory.getLogger(Cast.class); // TODO: Currently we only support top-level field casting. Ideally we could use a dotted notation in the spec to // allow casting nested fields. @@ -78,9 +86,16 @@ public String toString() { private static final String PURPOSE = "cast types"; - private static final Set SUPPORTED_CAST_TYPES = EnumSet.of( + private static final Set SUPPORTED_CAST_INPUT_TYPES = EnumSet.of( Schema.Type.INT8, Schema.Type.INT16, Schema.Type.INT32, Schema.Type.INT64, - Schema.Type.FLOAT32, Schema.Type.FLOAT64, Schema.Type.BOOLEAN, Schema.Type.STRING + Schema.Type.FLOAT32, Schema.Type.FLOAT64, Schema.Type.BOOLEAN, + Schema.Type.STRING, Schema.Type.BYTES + ); + + private static final Set SUPPORTED_CAST_OUTPUT_TYPES = EnumSet.of( + Schema.Type.INT8, Schema.Type.INT16, Schema.Type.INT32, Schema.Type.INT64, + Schema.Type.FLOAT32, Schema.Type.FLOAT64, Schema.Type.BOOLEAN, + Schema.Type.STRING ); // As a special case for casting the entire value (e.g. the incoming key is a int64 but you know it could be an @@ -120,14 +135,14 @@ public void close() { private R applySchemaless(R record) { if (wholeValueCastType != null) { - return newRecord(record, null, castValueToType(operatingValue(record), wholeValueCastType)); + return newRecord(record, null, castValueToType(null, operatingValue(record), wholeValueCastType)); } final Map value = requireMap(operatingValue(record), PURPOSE); final HashMap updatedValue = new HashMap<>(value); for (Map.Entry fieldSpec : casts.entrySet()) { String field = fieldSpec.getKey(); - updatedValue.put(field, castValueToType(value.get(field), fieldSpec.getValue())); + updatedValue.put(field, castValueToType(null, value.get(field), fieldSpec.getValue())); } return newRecord(record, null, updatedValue); } @@ -138,7 +153,7 @@ private R applyWithSchema(R record) { // Whole-record casting if (wholeValueCastType != null) - return newRecord(record, updatedSchema, castValueToType(operatingValue(record), wholeValueCastType)); + return newRecord(record, updatedSchema, castValueToType(valueSchema, operatingValue(record), wholeValueCastType)); // Casting within a struct final Struct value = requireStruct(operatingValue(record), PURPOSE); @@ -147,7 +162,8 @@ private R applyWithSchema(R record) { for (Field field : value.schema().fields()) { final Object origFieldValue = value.get(field); final Schema.Type targetType = casts.get(field.name()); - final Object newFieldValue = targetType != null ? castValueToType(origFieldValue, targetType) : origFieldValue; + final Object newFieldValue = targetType != null ? castValueToType(field.schema(), origFieldValue, targetType) : origFieldValue; + log.trace("Cast field '{}' from '{}' to '{}'", field.name(), origFieldValue, newFieldValue); updatedValue.put(updatedSchema.field(field.name()), newFieldValue); } return newRecord(record, updatedSchema, updatedValue); @@ -164,20 +180,26 @@ private Schema getOrBuildSchema(Schema valueSchema) { } else { builder = SchemaUtil.copySchemaBasics(valueSchema, SchemaBuilder.struct()); for (Field field : valueSchema.fields()) { - SchemaBuilder fieldBuilder = - convertFieldType(casts.containsKey(field.name()) ? casts.get(field.name()) : field.schema().type()); - if (field.schema().isOptional()) - fieldBuilder.optional(); - if (field.schema().defaultValue() != null) - fieldBuilder.defaultValue(castValueToType(field.schema().defaultValue(), fieldBuilder.type())); - builder.field(field.name(), fieldBuilder.build()); + if (casts.containsKey(field.name())) { + SchemaBuilder fieldBuilder = convertFieldType(casts.get(field.name())); + if (field.schema().isOptional()) + fieldBuilder.optional(); + if (field.schema().defaultValue() != null) { + Schema fieldSchema = field.schema(); + fieldBuilder.defaultValue(castValueToType(fieldSchema, fieldSchema.defaultValue(), fieldBuilder.type())); + } + builder.field(field.name(), fieldBuilder.build()); + } else { + builder.field(field.name(), field.schema()); + } + } } if (valueSchema.isOptional()) builder.optional(); if (valueSchema.defaultValue() != null) - builder.defaultValue(castValueToType(valueSchema.defaultValue(), builder.type())); + builder.defaultValue(castValueToType(valueSchema, valueSchema.defaultValue(), builder.type())); updatedSchema = builder.build(); schemaUpdateCache.put(valueSchema, updatedSchema); @@ -205,14 +227,26 @@ private SchemaBuilder convertFieldType(Schema.Type type) { default: throw new DataException("Unexpected type in Cast transformation: " + type); } + } + private static Object encodeLogicalType(Schema schema, Object value) { + switch (schema.name()) { + case Date.LOGICAL_NAME: + return Date.fromLogical(schema, (java.util.Date) value); + case Time.LOGICAL_NAME: + return Time.fromLogical(schema, (java.util.Date) value); + case Timestamp.LOGICAL_NAME: + return Timestamp.fromLogical(schema, (java.util.Date) value); + } + return value; } - private static Object castValueToType(Object value, Schema.Type targetType) { + private static Object castValueToType(Schema schema, Object value, Schema.Type targetType) { try { if (value == null) return null; - Schema.Type inferredType = ConnectSchema.schemaType(value.getClass()); + Schema.Type inferredType = schema == null ? ConnectSchema.schemaType(value.getClass()) : + schema.type(); if (inferredType == null) { throw new DataException("Cast transformation was passed a value of type " + value.getClass() + " which is not supported by Connect's data API"); @@ -220,6 +254,11 @@ private static Object castValueToType(Object value, Schema.Type targetType) { // Ensure the type we are trying to cast from is supported validCastType(inferredType, FieldType.INPUT); + // Perform logical type encoding to their internal representation. + if (schema != null && schema.name() != null && targetType != Type.STRING) { + value = encodeLogicalType(schema, value); + } + switch (targetType) { case INT8: return castToInt8(value); @@ -323,7 +362,12 @@ else if (value instanceof String) } private static String castToString(Object value) { - return value.toString(); + if (value instanceof java.util.Date) { + java.util.Date dateValue = (java.util.Date) value; + return Values.dateFormatFor(dateValue).format(dateValue); + } else { + return value.toString(); + } } protected abstract Schema operatingSchema(R record); @@ -366,15 +410,19 @@ private enum FieldType { } private static Schema.Type validCastType(Schema.Type type, FieldType fieldType) { - if (!SUPPORTED_CAST_TYPES.contains(type)) { - String message = "Cast transformation does not support casting to/from " + type - + "; supported types are " + SUPPORTED_CAST_TYPES; - switch (fieldType) { - case INPUT: - throw new DataException(message); - case OUTPUT: - throw new ConfigException(message); - } + switch (fieldType) { + case INPUT: + if (!SUPPORTED_CAST_INPUT_TYPES.contains(type)) { + throw new DataException("Cast transformation does not support casting from " + + type + "; supported types are " + SUPPORTED_CAST_INPUT_TYPES); + } + break; + case OUTPUT: + if (!SUPPORTED_CAST_OUTPUT_TYPES.contains(type)) { + throw new ConfigException("Cast transformation does not support casting to " + + type + "; supported types are " + SUPPORTED_CAST_OUTPUT_TYPES); + } + break; } return type; } diff --git a/connect/transforms/src/main/java/org/apache/kafka/connect/transforms/Flatten.java b/connect/transforms/src/main/java/org/apache/kafka/connect/transforms/Flatten.java index c5e4000975324..d7d21445d73d8 100644 --- a/connect/transforms/src/main/java/org/apache/kafka/connect/transforms/Flatten.java +++ b/connect/transforms/src/main/java/org/apache/kafka/connect/transforms/Flatten.java @@ -35,7 +35,7 @@ import java.util.Map; import static org.apache.kafka.connect.transforms.util.Requirements.requireMap; -import static org.apache.kafka.connect.transforms.util.Requirements.requireStruct; +import static org.apache.kafka.connect.transforms.util.Requirements.requireStructOrNull; public abstract class Flatten> implements Transformation { @@ -136,20 +136,24 @@ private void applySchemaless(Map originalRecord, String fieldNam } private R applyWithSchema(R record) { - final Struct value = requireStruct(operatingValue(record), PURPOSE); + final Struct value = requireStructOrNull(operatingValue(record), PURPOSE); - Schema updatedSchema = schemaUpdateCache.get(value.schema()); + Schema schema = operatingSchema(record); + Schema updatedSchema = schemaUpdateCache.get(schema); if (updatedSchema == null) { - final SchemaBuilder builder = SchemaUtil.copySchemaBasics(value.schema(), SchemaBuilder.struct()); - Struct defaultValue = (Struct) value.schema().defaultValue(); - buildUpdatedSchema(value.schema(), "", builder, value.schema().isOptional(), defaultValue); + final SchemaBuilder builder = SchemaUtil.copySchemaBasics(schema, SchemaBuilder.struct()); + Struct defaultValue = (Struct) schema.defaultValue(); + buildUpdatedSchema(schema, "", builder, schema.isOptional(), defaultValue); updatedSchema = builder.build(); - schemaUpdateCache.put(value.schema(), updatedSchema); + schemaUpdateCache.put(schema, updatedSchema); + } + if (value == null) { + return newRecord(record, updatedSchema, null); + } else { + final Struct updatedValue = new Struct(updatedSchema); + buildWithSchema(value, "", updatedValue); + return newRecord(record, updatedSchema, updatedValue); } - - final Struct updatedValue = new Struct(updatedSchema); - buildWithSchema(value, "", updatedValue); - return newRecord(record, updatedSchema, updatedValue); } /** @@ -216,6 +220,9 @@ private Schema convertFieldSchema(Schema orig, boolean optional, Object defaultF } private void buildWithSchema(Struct record, String fieldNamePrefix, Struct newRecord) { + if (record == null) { + return; + } for (Field field : record.schema().fields()) { final String fieldName = fieldName(fieldNamePrefix, field.name()); switch (field.schema().type()) { diff --git a/connect/transforms/src/main/java/org/apache/kafka/connect/transforms/InsertField.java b/connect/transforms/src/main/java/org/apache/kafka/connect/transforms/InsertField.java index 5e472a907c3f4..93ba79c0e3545 100644 --- a/connect/transforms/src/main/java/org/apache/kafka/connect/transforms/InsertField.java +++ b/connect/transforms/src/main/java/org/apache/kafka/connect/transforms/InsertField.java @@ -127,13 +127,19 @@ public void configure(Map props) { @Override public R apply(R record) { - if (operatingSchema(record) == null) { + if (isTombstoneRecord(record)) { + return record; + } else if (operatingSchema(record) == null) { return applySchemaless(record); } else { return applyWithSchema(record); } } + private boolean isTombstoneRecord(R record) { + return record.value() == null; + } + private R applySchemaless(R record) { final Map value = requireMap(operatingValue(record), PURPOSE); diff --git a/connect/transforms/src/main/java/org/apache/kafka/connect/transforms/SetSchemaMetadata.java b/connect/transforms/src/main/java/org/apache/kafka/connect/transforms/SetSchemaMetadata.java index 901ac9f18ad88..fd3cbf314ff5a 100644 --- a/connect/transforms/src/main/java/org/apache/kafka/connect/transforms/SetSchemaMetadata.java +++ b/connect/transforms/src/main/java/org/apache/kafka/connect/transforms/SetSchemaMetadata.java @@ -24,12 +24,15 @@ import org.apache.kafka.connect.data.Schema; import org.apache.kafka.connect.data.Struct; import org.apache.kafka.connect.transforms.util.SimpleConfig; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; import java.util.Map; import static org.apache.kafka.connect.transforms.util.Requirements.requireSchema; public abstract class SetSchemaMetadata> implements Transformation { + private static final Logger log = LoggerFactory.getLogger(SetSchemaMetadata.class); public static final String OVERVIEW_DOC = "Set the schema name, version or both on the record's key (" + Key.class.getName() + ")" @@ -76,6 +79,8 @@ public R apply(R record) { isMap ? schema.keySchema() : null, isMap || isArray ? schema.valueSchema() : null ); + log.trace("Applying SetSchemaMetadata SMT. Original schema: {}, updated schema: {}", + schema, updatedSchema); return newRecord(record, updatedSchema); } @@ -149,4 +154,4 @@ protected static Object updateSchemaIn(Object keyOrValue, Schema updatedSchema) } return keyOrValue; } -} \ No newline at end of file +} diff --git a/connect/transforms/src/main/java/org/apache/kafka/connect/transforms/TimestampConverter.java b/connect/transforms/src/main/java/org/apache/kafka/connect/transforms/TimestampConverter.java index 85574415f4559..f32253ec51b6d 100644 --- a/connect/transforms/src/main/java/org/apache/kafka/connect/transforms/TimestampConverter.java +++ b/connect/transforms/src/main/java/org/apache/kafka/connect/transforms/TimestampConverter.java @@ -47,7 +47,7 @@ import java.util.TimeZone; import static org.apache.kafka.connect.transforms.util.Requirements.requireMap; -import static org.apache.kafka.connect.transforms.util.Requirements.requireStruct; +import static org.apache.kafka.connect.transforms.util.Requirements.requireStructOrNull; public abstract class TimestampConverter> implements Transformation { @@ -85,6 +85,10 @@ public abstract class TimestampConverter> implements private static final TimeZone UTC = TimeZone.getTimeZone("UTC"); + public static final Schema OPTIONAL_DATE_SCHEMA = org.apache.kafka.connect.data.Date.builder().optional().schema(); + public static final Schema OPTIONAL_TIMESTAMP_SCHEMA = Timestamp.builder().optional().schema(); + public static final Schema OPTIONAL_TIME_SCHEMA = Time.builder().optional().schema(); + private interface TimestampTranslator { /** * Convert from the type-specific format to the universal java.util.Date format @@ -94,7 +98,7 @@ private interface TimestampTranslator { /** * Get the schema for this format. */ - Schema typeSchema(); + Schema typeSchema(boolean isOptional); /** * Convert from the universal java.util.Date format to the type-specific format @@ -118,8 +122,8 @@ public Date toRaw(Config config, Object orig) { } @Override - public Schema typeSchema() { - return Schema.STRING_SCHEMA; + public Schema typeSchema(boolean isOptional) { + return isOptional ? Schema.OPTIONAL_STRING_SCHEMA : Schema.STRING_SCHEMA; } @Override @@ -139,8 +143,8 @@ public Date toRaw(Config config, Object orig) { } @Override - public Schema typeSchema() { - return Schema.INT64_SCHEMA; + public Schema typeSchema(boolean isOptional) { + return isOptional ? Schema.OPTIONAL_INT64_SCHEMA : Schema.INT64_SCHEMA; } @Override @@ -159,8 +163,8 @@ public Date toRaw(Config config, Object orig) { } @Override - public Schema typeSchema() { - return org.apache.kafka.connect.data.Date.SCHEMA; + public Schema typeSchema(boolean isOptional) { + return isOptional ? OPTIONAL_DATE_SCHEMA : org.apache.kafka.connect.data.Date.SCHEMA; } @Override @@ -185,8 +189,8 @@ public Date toRaw(Config config, Object orig) { } @Override - public Schema typeSchema() { - return Time.SCHEMA; + public Schema typeSchema(boolean isOptional) { + return isOptional ? OPTIONAL_TIME_SCHEMA : Time.SCHEMA; } @Override @@ -212,8 +216,8 @@ public Date toRaw(Config config, Object orig) { } @Override - public Schema typeSchema() { - return Timestamp.SCHEMA; + public Schema typeSchema(boolean isOptional) { + return isOptional ? OPTIONAL_TIMESTAMP_SCHEMA : Timestamp.SCHEMA; } @Override @@ -330,16 +334,16 @@ private R applyWithSchema(R record) { if (config.field.isEmpty()) { Object value = operatingValue(record); // New schema is determined by the requested target timestamp type - Schema updatedSchema = TRANSLATORS.get(config.type).typeSchema(); + Schema updatedSchema = TRANSLATORS.get(config.type).typeSchema(schema.isOptional()); return newRecord(record, updatedSchema, convertTimestamp(value, timestampTypeFromSchema(schema))); } else { - final Struct value = requireStruct(operatingValue(record), PURPOSE); - Schema updatedSchema = schemaUpdateCache.get(value.schema()); + final Struct value = requireStructOrNull(operatingValue(record), PURPOSE); + Schema updatedSchema = schemaUpdateCache.get(schema); if (updatedSchema == null) { SchemaBuilder builder = SchemaUtil.copySchemaBasics(schema, SchemaBuilder.struct()); for (Field field : schema.fields()) { if (field.name().equals(config.field)) { - builder.field(field.name(), TRANSLATORS.get(config.type).typeSchema()); + builder.field(field.name(), TRANSLATORS.get(config.type).typeSchema(field.schema().isOptional())); } else { builder.field(field.name(), field.schema()); } @@ -361,6 +365,9 @@ private R applyWithSchema(R record) { } private Struct applyValueWithSchema(Struct value, Schema updatedSchema) { + if (value == null) { + return null; + } Struct updatedValue = new Struct(updatedSchema); for (Field field : value.schema().fields()) { final Object updatedFieldValue; @@ -375,11 +382,11 @@ private Struct applyValueWithSchema(Struct value, Schema updatedSchema) { } private R applySchemaless(R record) { - if (config.field.isEmpty()) { - Object value = operatingValue(record); - return newRecord(record, null, convertTimestamp(value)); + Object rawValue = operatingValue(record); + if (rawValue == null || config.field.isEmpty()) { + return newRecord(record, null, convertTimestamp(rawValue)); } else { - final Map value = requireMap(operatingValue(record), PURPOSE); + final Map value = requireMap(rawValue, PURPOSE); final HashMap updatedValue = new HashMap<>(value); updatedValue.put(config.field, convertTimestamp(value.get(config.field))); return newRecord(record, null, updatedValue); @@ -424,11 +431,14 @@ private String inferTimestampType(Object timestamp) { /** * Convert the given timestamp to the target timestamp format. - * @param timestamp the input timestamp + * @param timestamp the input timestamp, may be null * @param timestampFormat the format of the timestamp, or null if the format should be inferred * @return the converted timestamp */ private Object convertTimestamp(Object timestamp, String timestampFormat) { + if (timestamp == null) { + return null; + } if (timestampFormat == null) { timestampFormat = inferTimestampType(timestamp); } diff --git a/connect/transforms/src/test/java/org/apache/kafka/connect/transforms/CastTest.java b/connect/transforms/src/test/java/org/apache/kafka/connect/transforms/CastTest.java index b190189b35d9a..a28aa28c6d72e 100644 --- a/connect/transforms/src/test/java/org/apache/kafka/connect/transforms/CastTest.java +++ b/connect/transforms/src/test/java/org/apache/kafka/connect/transforms/CastTest.java @@ -17,16 +17,26 @@ package org.apache.kafka.connect.transforms; +import java.util.Arrays; +import java.util.List; +import java.util.concurrent.TimeUnit; import org.apache.kafka.common.config.ConfigException; +import org.apache.kafka.connect.data.Decimal; import org.apache.kafka.connect.data.Schema; +import org.apache.kafka.connect.data.Schema.Type; import org.apache.kafka.connect.data.SchemaBuilder; import org.apache.kafka.connect.data.Struct; +import org.apache.kafka.connect.data.Time; +import org.apache.kafka.connect.data.Timestamp; +import org.apache.kafka.connect.data.Values; import org.apache.kafka.connect.errors.DataException; import org.apache.kafka.connect.source.SourceRecord; import org.junit.After; import org.junit.Test; +import java.math.BigDecimal; import java.util.Collections; +import java.util.Date; import java.util.HashMap; import java.util.Map; @@ -37,6 +47,8 @@ public class CastTest { private final Cast xformKey = new Cast.Key<>(); private final Cast xformValue = new Cast.Value<>(); + private static final long MILLIS_PER_HOUR = TimeUnit.HOURS.toMillis(1); + private static final long MILLIS_PER_DAY = TimeUnit.DAYS.toMillis(1); @After public void teardown() { @@ -59,6 +71,11 @@ public void testConfigInvalidTargetType() { xformKey.configure(Collections.singletonMap(Cast.SPEC_CONFIG, "foo:array")); } + @Test(expected = ConfigException.class) + public void testUnsupportedTargetType() { + xformKey.configure(Collections.singletonMap(Cast.SPEC_CONFIG, "foo:bytes")); + } + @Test(expected = ConfigException.class) public void testConfigInvalidMap() { xformKey.configure(Collections.singletonMap(Cast.SPEC_CONFIG, "foo:int8:extra")); @@ -169,6 +186,28 @@ public void castWholeRecordValueWithSchemaString() { assertEquals("42", transformed.value()); } + @Test + public void castWholeBigDecimalRecordValueWithSchemaString() { + BigDecimal bigDecimal = new BigDecimal(42); + xformValue.configure(Collections.singletonMap(Cast.SPEC_CONFIG, "string")); + SourceRecord transformed = xformValue.apply(new SourceRecord(null, null, "topic", 0, + Decimal.schema(bigDecimal.scale()), bigDecimal)); + + assertEquals(Schema.Type.STRING, transformed.valueSchema().type()); + assertEquals("42", transformed.value()); + } + + @Test + public void castWholeDateRecordValueWithSchemaString() { + Date timestamp = new Date(MILLIS_PER_DAY + 1); // day + 1msec to get a timestamp formatting. + xformValue.configure(Collections.singletonMap(Cast.SPEC_CONFIG, "string")); + SourceRecord transformed = xformValue.apply(new SourceRecord(null, null, "topic", 0, + Timestamp.SCHEMA, timestamp)); + + assertEquals(Schema.Type.STRING, transformed.valueSchema().type()); + assertEquals(Values.dateFormatFor(timestamp).format(timestamp), transformed.value()); + } + @Test public void castWholeRecordDefaultValue() { // Validate default value in schema is correctly converted @@ -287,10 +326,102 @@ public void castWholeRecordValueSchemalessUnsupportedType() { xformValue.apply(new SourceRecord(null, null, "topic", 0, null, Collections.singletonList("foo"))); } + @Test + public void castLogicalToPrimitive() { + List specParts = Arrays.asList( + "date_to_int32:int32", // Cast to underlying representation + "timestamp_to_int64:int64", // Cast to underlying representation + "time_to_int64:int64", // Cast to wider datatype than underlying representation + "decimal_to_int32:int32", // Cast to narrower datatype with data loss + "timestamp_to_float64:float64", // loss of precision casting to double + "null_timestamp_to_int32:int32" + ); + + Date day = new Date(MILLIS_PER_DAY); + xformValue.configure(Collections.singletonMap(Cast.SPEC_CONFIG, + String.join(",", specParts))); + + SchemaBuilder builder = SchemaBuilder.struct(); + builder.field("date_to_int32", org.apache.kafka.connect.data.Date.SCHEMA); + builder.field("timestamp_to_int64", Timestamp.SCHEMA); + builder.field("time_to_int64", Time.SCHEMA); + builder.field("decimal_to_int32", Decimal.schema(new BigDecimal((long) Integer.MAX_VALUE + 1).scale())); + builder.field("timestamp_to_float64", Timestamp.SCHEMA); + builder.field("null_timestamp_to_int32", Timestamp.builder().optional().build()); + + Schema supportedTypesSchema = builder.build(); + + Struct recordValue = new Struct(supportedTypesSchema); + recordValue.put("date_to_int32", day); + recordValue.put("timestamp_to_int64", new Date(0)); + recordValue.put("time_to_int64", new Date(1)); + recordValue.put("decimal_to_int32", new BigDecimal((long) Integer.MAX_VALUE + 1)); + recordValue.put("timestamp_to_float64", new Date(Long.MAX_VALUE)); + recordValue.put("null_timestamp_to_int32", null); + + SourceRecord transformed = xformValue.apply( + new SourceRecord(null, null, "topic", 0, + supportedTypesSchema, recordValue)); + + assertEquals(1, ((Struct) transformed.value()).get("date_to_int32")); + assertEquals(0L, ((Struct) transformed.value()).get("timestamp_to_int64")); + assertEquals(1L, ((Struct) transformed.value()).get("time_to_int64")); + assertEquals(Integer.MIN_VALUE, ((Struct) transformed.value()).get("decimal_to_int32")); + assertEquals(9.223372036854776E18, ((Struct) transformed.value()).get("timestamp_to_float64")); + assertNull(((Struct) transformed.value()).get("null_timestamp_to_int32")); + + Schema transformedSchema = ((Struct) transformed.value()).schema(); + assertEquals(Type.INT32, transformedSchema.field("date_to_int32").schema().type()); + assertEquals(Type.INT64, transformedSchema.field("timestamp_to_int64").schema().type()); + assertEquals(Type.INT64, transformedSchema.field("time_to_int64").schema().type()); + assertEquals(Type.INT32, transformedSchema.field("decimal_to_int32").schema().type()); + assertEquals(Type.FLOAT64, transformedSchema.field("timestamp_to_float64").schema().type()); + assertEquals(Type.INT32, transformedSchema.field("null_timestamp_to_int32").schema().type()); + } + + @Test + public void castLogicalToString() { + Date date = new Date(MILLIS_PER_DAY); + Date time = new Date(MILLIS_PER_HOUR); + Date timestamp = new Date(); + + xformValue.configure(Collections.singletonMap(Cast.SPEC_CONFIG, + "date:string,decimal:string,time:string,timestamp:string")); + + SchemaBuilder builder = SchemaBuilder.struct(); + builder.field("date", org.apache.kafka.connect.data.Date.SCHEMA); + builder.field("decimal", Decimal.schema(new BigDecimal(1982).scale())); + builder.field("time", Time.SCHEMA); + builder.field("timestamp", Timestamp.SCHEMA); + + Schema supportedTypesSchema = builder.build(); + + Struct recordValue = new Struct(supportedTypesSchema); + recordValue.put("date", date); + recordValue.put("decimal", new BigDecimal(1982)); + recordValue.put("time", time); + recordValue.put("timestamp", timestamp); + + SourceRecord transformed = xformValue.apply( + new SourceRecord(null, null, "topic", 0, + supportedTypesSchema, recordValue)); + + assertEquals(Values.dateFormatFor(date).format(date), ((Struct) transformed.value()).get("date")); + assertEquals("1982", ((Struct) transformed.value()).get("decimal")); + assertEquals(Values.dateFormatFor(time).format(time), ((Struct) transformed.value()).get("time")); + assertEquals(Values.dateFormatFor(timestamp).format(timestamp), ((Struct) transformed.value()).get("timestamp")); + + Schema transformedSchema = ((Struct) transformed.value()).schema(); + assertEquals(Type.STRING, transformedSchema.field("date").schema().type()); + assertEquals(Type.STRING, transformedSchema.field("decimal").schema().type()); + assertEquals(Type.STRING, transformedSchema.field("time").schema().type()); + assertEquals(Type.STRING, transformedSchema.field("timestamp").schema().type()); + } @Test public void castFieldsWithSchema() { - xformValue.configure(Collections.singletonMap(Cast.SPEC_CONFIG, "int8:int16,int16:int32,int32:int64,int64:boolean,float32:float64,float64:boolean,boolean:int8,string:int32,optional:int32")); + Date day = new Date(MILLIS_PER_DAY); + xformValue.configure(Collections.singletonMap(Cast.SPEC_CONFIG, "int8:int16,int16:int32,int32:int64,int64:boolean,float32:float64,float64:boolean,boolean:int8,string:int32,bigdecimal:string,date:string,optional:int32")); // Include an optional fields and fields with defaults to validate their values are passed through properly SchemaBuilder builder = SchemaBuilder.struct(); @@ -303,7 +434,10 @@ public void castFieldsWithSchema() { builder.field("float64", SchemaBuilder.float64().defaultValue(-1.125).build()); builder.field("boolean", Schema.BOOLEAN_SCHEMA); builder.field("string", Schema.STRING_SCHEMA); + builder.field("bigdecimal", Decimal.schema(new BigDecimal(42).scale())); + builder.field("date", org.apache.kafka.connect.data.Date.SCHEMA); builder.field("optional", Schema.OPTIONAL_FLOAT32_SCHEMA); + builder.field("timestamp", Timestamp.SCHEMA); Schema supportedTypesSchema = builder.build(); Struct recordValue = new Struct(supportedTypesSchema); @@ -314,7 +448,10 @@ public void castFieldsWithSchema() { recordValue.put("float32", 32.f); recordValue.put("float64", -64.); recordValue.put("boolean", true); + recordValue.put("bigdecimal", new BigDecimal(42)); + recordValue.put("date", day); recordValue.put("string", "42"); + recordValue.put("timestamp", new Date(0)); // optional field intentionally omitted SourceRecord transformed = xformValue.apply(new SourceRecord(null, null, "topic", 0, @@ -331,7 +468,25 @@ public void castFieldsWithSchema() { assertEquals(true, ((Struct) transformed.value()).schema().field("float64").schema().defaultValue()); assertEquals((byte) 1, ((Struct) transformed.value()).get("boolean")); assertEquals(42, ((Struct) transformed.value()).get("string")); + assertEquals("42", ((Struct) transformed.value()).get("bigdecimal")); + assertEquals(Values.dateFormatFor(day).format(day), ((Struct) transformed.value()).get("date")); + assertEquals(new Date(0), ((Struct) transformed.value()).get("timestamp")); assertNull(((Struct) transformed.value()).get("optional")); + + Schema transformedSchema = ((Struct) transformed.value()).schema(); + assertEquals(Schema.INT16_SCHEMA.type(), transformedSchema.field("int8").schema().type()); + assertEquals(Schema.OPTIONAL_INT32_SCHEMA.type(), transformedSchema.field("int16").schema().type()); + assertEquals(Schema.INT64_SCHEMA.type(), transformedSchema.field("int32").schema().type()); + assertEquals(Schema.BOOLEAN_SCHEMA.type(), transformedSchema.field("int64").schema().type()); + assertEquals(Schema.FLOAT64_SCHEMA.type(), transformedSchema.field("float32").schema().type()); + assertEquals(Schema.BOOLEAN_SCHEMA.type(), transformedSchema.field("float64").schema().type()); + assertEquals(Schema.INT8_SCHEMA.type(), transformedSchema.field("boolean").schema().type()); + assertEquals(Schema.INT32_SCHEMA.type(), transformedSchema.field("string").schema().type()); + assertEquals(Schema.STRING_SCHEMA.type(), transformedSchema.field("bigdecimal").schema().type()); + assertEquals(Schema.STRING_SCHEMA.type(), transformedSchema.field("date").schema().type()); + assertEquals(Schema.OPTIONAL_INT32_SCHEMA.type(), transformedSchema.field("optional").schema().type()); + // The following fields are not changed + assertEquals(Timestamp.SCHEMA.type(), transformedSchema.field("timestamp").schema().type()); } @SuppressWarnings("unchecked") diff --git a/connect/transforms/src/test/java/org/apache/kafka/connect/transforms/FlattenTest.java b/connect/transforms/src/test/java/org/apache/kafka/connect/transforms/FlattenTest.java index d709054dbe043..430bba6f64022 100644 --- a/connect/transforms/src/test/java/org/apache/kafka/connect/transforms/FlattenTest.java +++ b/connect/transforms/src/test/java/org/apache/kafka/connect/transforms/FlattenTest.java @@ -181,6 +181,46 @@ public void testOptionalFieldStruct() { assertNull(transformedStruct.get("B.opt_int32")); } + @Test + public void testOptionalStruct() { + xformValue.configure(Collections.emptyMap()); + + SchemaBuilder builder = SchemaBuilder.struct().optional(); + builder.field("opt_int32", Schema.OPTIONAL_INT32_SCHEMA); + Schema schema = builder.build(); + + SourceRecord transformed = xformValue.apply(new SourceRecord(null, null, + "topic", 0, + schema, null)); + + assertEquals(Schema.Type.STRUCT, transformed.valueSchema().type()); + assertNull(transformed.value()); + } + + @Test + public void testOptionalNestedStruct() { + xformValue.configure(Collections.emptyMap()); + + SchemaBuilder builder = SchemaBuilder.struct().optional(); + builder.field("opt_int32", Schema.OPTIONAL_INT32_SCHEMA); + Schema supportedTypesSchema = builder.build(); + + builder = SchemaBuilder.struct(); + builder.field("B", supportedTypesSchema); + Schema oneLevelNestedSchema = builder.build(); + + Struct oneLevelNestedStruct = new Struct(oneLevelNestedSchema); + oneLevelNestedStruct.put("B", null); + + SourceRecord transformed = xformValue.apply(new SourceRecord(null, null, + "topic", 0, + oneLevelNestedSchema, oneLevelNestedStruct)); + + assertEquals(Schema.Type.STRUCT, transformed.valueSchema().type()); + Struct transformedStruct = (Struct) transformed.value(); + assertNull(transformedStruct.get("B.opt_int32")); + } + @Test public void testOptionalFieldMap() { xformValue.configure(Collections.emptyMap()); diff --git a/connect/transforms/src/test/java/org/apache/kafka/connect/transforms/InsertFieldTest.java b/connect/transforms/src/test/java/org/apache/kafka/connect/transforms/InsertFieldTest.java index a0a09752a4ae9..b22872cacb236 100644 --- a/connect/transforms/src/test/java/org/apache/kafka/connect/transforms/InsertFieldTest.java +++ b/connect/transforms/src/test/java/org/apache/kafka/connect/transforms/InsertFieldTest.java @@ -104,11 +104,53 @@ public void schemalessInsertConfiguredFields() { final SourceRecord transformedRecord = xform.apply(record); - assertEquals(42L, ((Map) transformedRecord.value()).get("magic")); - assertEquals("test", ((Map) transformedRecord.value()).get("topic_field")); - assertEquals(0, ((Map) transformedRecord.value()).get("partition_field")); - assertEquals(null, ((Map) transformedRecord.value()).get("timestamp_field")); - assertEquals("my-instance-id", ((Map) transformedRecord.value()).get("instance_id")); + assertEquals(42L, ((Map) transformedRecord.value()).get("magic")); + assertEquals("test", ((Map) transformedRecord.value()).get("topic_field")); + assertEquals(0, ((Map) transformedRecord.value()).get("partition_field")); + assertEquals(null, ((Map) transformedRecord.value()).get("timestamp_field")); + assertEquals("my-instance-id", ((Map) transformedRecord.value()).get("instance_id")); } + + @Test + public void insertConfiguredFieldsIntoTombstoneEventWithoutSchemaLeavesValueUnchanged() { + final Map props = new HashMap<>(); + props.put("topic.field", "topic_field!"); + props.put("partition.field", "partition_field"); + props.put("timestamp.field", "timestamp_field?"); + props.put("static.field", "instance_id"); + props.put("static.value", "my-instance-id"); + + xform.configure(props); + + final SourceRecord record = new SourceRecord(null, null, "test", 0, + null, null); + + final SourceRecord transformedRecord = xform.apply(record); + + assertEquals(null, transformedRecord.value()); + assertEquals(null, transformedRecord.valueSchema()); + } + + @Test + public void insertConfiguredFieldsIntoTombstoneEventWithSchemaLeavesValueUnchanged() { + final Map props = new HashMap<>(); + props.put("topic.field", "topic_field!"); + props.put("partition.field", "partition_field"); + props.put("timestamp.field", "timestamp_field?"); + props.put("static.field", "instance_id"); + props.put("static.value", "my-instance-id"); + + xform.configure(props); + + final Schema simpleStructSchema = SchemaBuilder.struct().name("name").version(1).doc("doc").field("magic", Schema.OPTIONAL_INT64_SCHEMA).build(); + + final SourceRecord record = new SourceRecord(null, null, "test", 0, + simpleStructSchema, null); + + final SourceRecord transformedRecord = xform.apply(record); + + assertEquals(null, transformedRecord.value()); + assertEquals(simpleStructSchema, transformedRecord.valueSchema()); + } } diff --git a/connect/transforms/src/test/java/org/apache/kafka/connect/transforms/TimestampConverterTest.java b/connect/transforms/src/test/java/org/apache/kafka/connect/transforms/TimestampConverterTest.java index 475066f74e281..3a1920ed528ce 100644 --- a/connect/transforms/src/test/java/org/apache/kafka/connect/transforms/TimestampConverterTest.java +++ b/connect/transforms/src/test/java/org/apache/kafka/connect/transforms/TimestampConverterTest.java @@ -35,6 +35,7 @@ import java.util.Map; import java.util.TimeZone; +import static org.apache.kafka.connect.transforms.util.Requirements.requireStruct; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertNull; @@ -105,13 +106,12 @@ public void testConfigInvalidFormat() { xformValue.configure(config); } - // Conversions without schemas (most flexible Timestamp -> other types) @Test public void testSchemalessIdentity() { xformValue.configure(Collections.singletonMap(TimestampConverter.TARGET_TYPE_CONFIG, "Timestamp")); - SourceRecord transformed = xformValue.apply(new SourceRecord(null, null, "topic", 0, null, DATE_PLUS_TIME.getTime())); + SourceRecord transformed = xformValue.apply(createRecordSchemaless(DATE_PLUS_TIME.getTime())); assertNull(transformed.valueSchema()); assertEquals(DATE_PLUS_TIME.getTime(), transformed.value()); @@ -120,7 +120,7 @@ public void testSchemalessIdentity() { @Test public void testSchemalessTimestampToDate() { xformValue.configure(Collections.singletonMap(TimestampConverter.TARGET_TYPE_CONFIG, "Date")); - SourceRecord transformed = xformValue.apply(new SourceRecord(null, null, "topic", 0, null, DATE_PLUS_TIME.getTime())); + SourceRecord transformed = xformValue.apply(createRecordSchemaless(DATE_PLUS_TIME.getTime())); assertNull(transformed.valueSchema()); assertEquals(DATE.getTime(), transformed.value()); @@ -129,7 +129,7 @@ public void testSchemalessTimestampToDate() { @Test public void testSchemalessTimestampToTime() { xformValue.configure(Collections.singletonMap(TimestampConverter.TARGET_TYPE_CONFIG, "Time")); - SourceRecord transformed = xformValue.apply(new SourceRecord(null, null, "topic", 0, null, DATE_PLUS_TIME.getTime())); + SourceRecord transformed = xformValue.apply(createRecordSchemaless(DATE_PLUS_TIME.getTime())); assertNull(transformed.valueSchema()); assertEquals(TIME.getTime(), transformed.value()); @@ -138,7 +138,7 @@ public void testSchemalessTimestampToTime() { @Test public void testSchemalessTimestampToUnix() { xformValue.configure(Collections.singletonMap(TimestampConverter.TARGET_TYPE_CONFIG, "unix")); - SourceRecord transformed = xformValue.apply(new SourceRecord(null, null, "topic", 0, null, DATE_PLUS_TIME.getTime())); + SourceRecord transformed = xformValue.apply(createRecordSchemaless(DATE_PLUS_TIME.getTime())); assertNull(transformed.valueSchema()); assertEquals(DATE_PLUS_TIME_UNIX, transformed.value()); @@ -150,7 +150,7 @@ public void testSchemalessTimestampToString() { config.put(TimestampConverter.TARGET_TYPE_CONFIG, "string"); config.put(TimestampConverter.FORMAT_CONFIG, STRING_DATE_FMT); xformValue.configure(config); - SourceRecord transformed = xformValue.apply(new SourceRecord(null, null, "topic", 0, null, DATE_PLUS_TIME.getTime())); + SourceRecord transformed = xformValue.apply(createRecordSchemaless(DATE_PLUS_TIME.getTime())); assertNull(transformed.valueSchema()); assertEquals(DATE_PLUS_TIME_STRING, transformed.value()); @@ -162,7 +162,7 @@ public void testSchemalessTimestampToString() { @Test public void testSchemalessDateToTimestamp() { xformValue.configure(Collections.singletonMap(TimestampConverter.TARGET_TYPE_CONFIG, "Timestamp")); - SourceRecord transformed = xformValue.apply(new SourceRecord(null, null, "topic", 0, null, DATE.getTime())); + SourceRecord transformed = xformValue.apply(createRecordSchemaless(DATE.getTime())); assertNull(transformed.valueSchema()); // No change expected since the source type is coarser-grained @@ -172,7 +172,7 @@ public void testSchemalessDateToTimestamp() { @Test public void testSchemalessTimeToTimestamp() { xformValue.configure(Collections.singletonMap(TimestampConverter.TARGET_TYPE_CONFIG, "Timestamp")); - SourceRecord transformed = xformValue.apply(new SourceRecord(null, null, "topic", 0, null, TIME.getTime())); + SourceRecord transformed = xformValue.apply(createRecordSchemaless(TIME.getTime())); assertNull(transformed.valueSchema()); // No change expected since the source type is coarser-grained @@ -182,7 +182,7 @@ public void testSchemalessTimeToTimestamp() { @Test public void testSchemalessUnixToTimestamp() { xformValue.configure(Collections.singletonMap(TimestampConverter.TARGET_TYPE_CONFIG, "Timestamp")); - SourceRecord transformed = xformValue.apply(new SourceRecord(null, null, "topic", 0, null, DATE_PLUS_TIME_UNIX)); + SourceRecord transformed = xformValue.apply(createRecordSchemaless(DATE_PLUS_TIME_UNIX)); assertNull(transformed.valueSchema()); assertEquals(DATE_PLUS_TIME.getTime(), transformed.value()); @@ -194,7 +194,7 @@ public void testSchemalessStringToTimestamp() { config.put(TimestampConverter.TARGET_TYPE_CONFIG, "Timestamp"); config.put(TimestampConverter.FORMAT_CONFIG, STRING_DATE_FMT); xformValue.configure(config); - SourceRecord transformed = xformValue.apply(new SourceRecord(null, null, "topic", 0, null, DATE_PLUS_TIME_STRING)); + SourceRecord transformed = xformValue.apply(createRecordSchemaless(DATE_PLUS_TIME_STRING)); assertNull(transformed.valueSchema()); assertEquals(DATE_PLUS_TIME.getTime(), transformed.value()); @@ -206,7 +206,7 @@ public void testSchemalessStringToTimestamp() { @Test public void testWithSchemaIdentity() { xformValue.configure(Collections.singletonMap(TimestampConverter.TARGET_TYPE_CONFIG, "Timestamp")); - SourceRecord transformed = xformValue.apply(new SourceRecord(null, null, "topic", 0, Timestamp.SCHEMA, DATE_PLUS_TIME.getTime())); + SourceRecord transformed = xformValue.apply(createRecordWithSchema(Timestamp.SCHEMA, DATE_PLUS_TIME.getTime())); assertEquals(Timestamp.SCHEMA, transformed.valueSchema()); assertEquals(DATE_PLUS_TIME.getTime(), transformed.value()); @@ -215,7 +215,7 @@ public void testWithSchemaIdentity() { @Test public void testWithSchemaTimestampToDate() { xformValue.configure(Collections.singletonMap(TimestampConverter.TARGET_TYPE_CONFIG, "Date")); - SourceRecord transformed = xformValue.apply(new SourceRecord(null, null, "topic", 0, Timestamp.SCHEMA, DATE_PLUS_TIME.getTime())); + SourceRecord transformed = xformValue.apply(createRecordWithSchema(Timestamp.SCHEMA, DATE_PLUS_TIME.getTime())); assertEquals(Date.SCHEMA, transformed.valueSchema()); assertEquals(DATE.getTime(), transformed.value()); @@ -224,7 +224,7 @@ public void testWithSchemaTimestampToDate() { @Test public void testWithSchemaTimestampToTime() { xformValue.configure(Collections.singletonMap(TimestampConverter.TARGET_TYPE_CONFIG, "Time")); - SourceRecord transformed = xformValue.apply(new SourceRecord(null, null, "topic", 0, Timestamp.SCHEMA, DATE_PLUS_TIME.getTime())); + SourceRecord transformed = xformValue.apply(createRecordWithSchema(Timestamp.SCHEMA, DATE_PLUS_TIME.getTime())); assertEquals(Time.SCHEMA, transformed.valueSchema()); assertEquals(TIME.getTime(), transformed.value()); @@ -233,7 +233,7 @@ public void testWithSchemaTimestampToTime() { @Test public void testWithSchemaTimestampToUnix() { xformValue.configure(Collections.singletonMap(TimestampConverter.TARGET_TYPE_CONFIG, "unix")); - SourceRecord transformed = xformValue.apply(new SourceRecord(null, null, "topic", 0, Timestamp.SCHEMA, DATE_PLUS_TIME.getTime())); + SourceRecord transformed = xformValue.apply(createRecordWithSchema(Timestamp.SCHEMA, DATE_PLUS_TIME.getTime())); assertEquals(Schema.INT64_SCHEMA, transformed.valueSchema()); assertEquals(DATE_PLUS_TIME_UNIX, transformed.value()); @@ -245,19 +245,70 @@ public void testWithSchemaTimestampToString() { config.put(TimestampConverter.TARGET_TYPE_CONFIG, "string"); config.put(TimestampConverter.FORMAT_CONFIG, STRING_DATE_FMT); xformValue.configure(config); - SourceRecord transformed = xformValue.apply(new SourceRecord(null, null, "topic", 0, Timestamp.SCHEMA, DATE_PLUS_TIME.getTime())); + SourceRecord transformed = xformValue.apply(createRecordWithSchema(Timestamp.SCHEMA, DATE_PLUS_TIME.getTime())); assertEquals(Schema.STRING_SCHEMA, transformed.valueSchema()); assertEquals(DATE_PLUS_TIME_STRING, transformed.value()); } + // Null-value conversions schemaless + + @Test + public void testSchemalessNullValueToString() { + testSchemalessNullValueConversion("string"); + testSchemalessNullFieldConversion("string"); + } + @Test + public void testSchemalessNullValueToDate() { + testSchemalessNullValueConversion("Date"); + testSchemalessNullFieldConversion("Date"); + } + @Test + public void testSchemalessNullValueToTimestamp() { + testSchemalessNullValueConversion("Timestamp"); + testSchemalessNullFieldConversion("Timestamp"); + } + @Test + public void testSchemalessNullValueToUnix() { + testSchemalessNullValueConversion("unix"); + testSchemalessNullFieldConversion("unix"); + } + + @Test + public void testSchemalessNullValueToTime() { + testSchemalessNullValueConversion("Time"); + testSchemalessNullFieldConversion("Time"); + } + + private void testSchemalessNullValueConversion(String targetType) { + Map config = new HashMap<>(); + config.put(TimestampConverter.TARGET_TYPE_CONFIG, targetType); + config.put(TimestampConverter.FORMAT_CONFIG, STRING_DATE_FMT); + xformValue.configure(config); + SourceRecord transformed = xformValue.apply(createRecordSchemaless(null)); + + assertNull(transformed.valueSchema()); + assertNull(transformed.value()); + } + + private void testSchemalessNullFieldConversion(String targetType) { + Map config = new HashMap<>(); + config.put(TimestampConverter.TARGET_TYPE_CONFIG, targetType); + config.put(TimestampConverter.FORMAT_CONFIG, STRING_DATE_FMT); + config.put(TimestampConverter.FIELD_CONFIG, "ts"); + xformValue.configure(config); + SourceRecord transformed = xformValue.apply(createRecordSchemaless(null)); + + assertNull(transformed.valueSchema()); + assertNull(transformed.value()); + } // Conversions with schemas (core types -> most flexible Timestamp format) @Test public void testWithSchemaDateToTimestamp() { xformValue.configure(Collections.singletonMap(TimestampConverter.TARGET_TYPE_CONFIG, "Timestamp")); - SourceRecord transformed = xformValue.apply(new SourceRecord(null, null, "topic", 0, Date.SCHEMA, DATE.getTime())); + SourceRecord transformed = xformValue.apply(createRecordWithSchema(Date.SCHEMA, DATE.getTime())); assertEquals(Timestamp.SCHEMA, transformed.valueSchema()); // No change expected since the source type is coarser-grained @@ -267,7 +318,7 @@ public void testWithSchemaDateToTimestamp() { @Test public void testWithSchemaTimeToTimestamp() { xformValue.configure(Collections.singletonMap(TimestampConverter.TARGET_TYPE_CONFIG, "Timestamp")); - SourceRecord transformed = xformValue.apply(new SourceRecord(null, null, "topic", 0, Time.SCHEMA, TIME.getTime())); + SourceRecord transformed = xformValue.apply(createRecordWithSchema(Time.SCHEMA, TIME.getTime())); assertEquals(Timestamp.SCHEMA, transformed.valueSchema()); // No change expected since the source type is coarser-grained @@ -277,7 +328,7 @@ public void testWithSchemaTimeToTimestamp() { @Test public void testWithSchemaUnixToTimestamp() { xformValue.configure(Collections.singletonMap(TimestampConverter.TARGET_TYPE_CONFIG, "Timestamp")); - SourceRecord transformed = xformValue.apply(new SourceRecord(null, null, "topic", 0, Schema.INT64_SCHEMA, DATE_PLUS_TIME_UNIX)); + SourceRecord transformed = xformValue.apply(createRecordWithSchema(Schema.INT64_SCHEMA, DATE_PLUS_TIME_UNIX)); assertEquals(Timestamp.SCHEMA, transformed.valueSchema()); assertEquals(DATE_PLUS_TIME.getTime(), transformed.value()); @@ -289,12 +340,145 @@ public void testWithSchemaStringToTimestamp() { config.put(TimestampConverter.TARGET_TYPE_CONFIG, "Timestamp"); config.put(TimestampConverter.FORMAT_CONFIG, STRING_DATE_FMT); xformValue.configure(config); - SourceRecord transformed = xformValue.apply(new SourceRecord(null, null, "topic", 0, Schema.STRING_SCHEMA, DATE_PLUS_TIME_STRING)); + SourceRecord transformed = xformValue.apply(createRecordWithSchema(Schema.STRING_SCHEMA, DATE_PLUS_TIME_STRING)); assertEquals(Timestamp.SCHEMA, transformed.valueSchema()); assertEquals(DATE_PLUS_TIME.getTime(), transformed.value()); } + // Null-value conversions with schema + + @Test + public void testWithSchemaNullValueToTimestamp() { + testWithSchemaNullValueConversion("Timestamp", Schema.OPTIONAL_INT64_SCHEMA, TimestampConverter.OPTIONAL_TIMESTAMP_SCHEMA); + testWithSchemaNullValueConversion("Timestamp", TimestampConverter.OPTIONAL_TIME_SCHEMA, TimestampConverter.OPTIONAL_TIMESTAMP_SCHEMA); + testWithSchemaNullValueConversion("Timestamp", TimestampConverter.OPTIONAL_DATE_SCHEMA, TimestampConverter.OPTIONAL_TIMESTAMP_SCHEMA); + testWithSchemaNullValueConversion("Timestamp", Schema.OPTIONAL_STRING_SCHEMA, TimestampConverter.OPTIONAL_TIMESTAMP_SCHEMA); + testWithSchemaNullValueConversion("Timestamp", TimestampConverter.OPTIONAL_TIMESTAMP_SCHEMA, TimestampConverter.OPTIONAL_TIMESTAMP_SCHEMA); + } + + @Test + public void testWithSchemaNullFieldToTimestamp() { + testWithSchemaNullFieldConversion("Timestamp", Schema.OPTIONAL_INT64_SCHEMA, TimestampConverter.OPTIONAL_TIMESTAMP_SCHEMA); + testWithSchemaNullFieldConversion("Timestamp", TimestampConverter.OPTIONAL_TIME_SCHEMA, TimestampConverter.OPTIONAL_TIMESTAMP_SCHEMA); + testWithSchemaNullFieldConversion("Timestamp", TimestampConverter.OPTIONAL_DATE_SCHEMA, TimestampConverter.OPTIONAL_TIMESTAMP_SCHEMA); + testWithSchemaNullFieldConversion("Timestamp", Schema.OPTIONAL_STRING_SCHEMA, TimestampConverter.OPTIONAL_TIMESTAMP_SCHEMA); + testWithSchemaNullFieldConversion("Timestamp", TimestampConverter.OPTIONAL_TIMESTAMP_SCHEMA, TimestampConverter.OPTIONAL_TIMESTAMP_SCHEMA); + } + + @Test + public void testWithSchemaNullValueToUnix() { + testWithSchemaNullValueConversion("unix", Schema.OPTIONAL_INT64_SCHEMA, Schema.OPTIONAL_INT64_SCHEMA); + testWithSchemaNullValueConversion("unix", TimestampConverter.OPTIONAL_TIME_SCHEMA, Schema.OPTIONAL_INT64_SCHEMA); + testWithSchemaNullValueConversion("unix", TimestampConverter.OPTIONAL_DATE_SCHEMA, Schema.OPTIONAL_INT64_SCHEMA); + testWithSchemaNullValueConversion("unix", Schema.OPTIONAL_STRING_SCHEMA, Schema.OPTIONAL_INT64_SCHEMA); + testWithSchemaNullValueConversion("unix", TimestampConverter.OPTIONAL_TIMESTAMP_SCHEMA, Schema.OPTIONAL_INT64_SCHEMA); + } + + @Test + public void testWithSchemaNullFieldToUnix() { + testWithSchemaNullFieldConversion("unix", Schema.OPTIONAL_INT64_SCHEMA, Schema.OPTIONAL_INT64_SCHEMA); + testWithSchemaNullFieldConversion("unix", TimestampConverter.OPTIONAL_TIME_SCHEMA, Schema.OPTIONAL_INT64_SCHEMA); + testWithSchemaNullFieldConversion("unix", TimestampConverter.OPTIONAL_DATE_SCHEMA, Schema.OPTIONAL_INT64_SCHEMA); + testWithSchemaNullFieldConversion("unix", Schema.OPTIONAL_STRING_SCHEMA, Schema.OPTIONAL_INT64_SCHEMA); + testWithSchemaNullFieldConversion("unix", TimestampConverter.OPTIONAL_TIMESTAMP_SCHEMA, Schema.OPTIONAL_INT64_SCHEMA); + } + + @Test + public void testWithSchemaNullValueToTime() { + testWithSchemaNullValueConversion("Time", Schema.OPTIONAL_INT64_SCHEMA, TimestampConverter.OPTIONAL_TIME_SCHEMA); + testWithSchemaNullValueConversion("Time", TimestampConverter.OPTIONAL_TIME_SCHEMA, TimestampConverter.OPTIONAL_TIME_SCHEMA); + testWithSchemaNullValueConversion("Time", TimestampConverter.OPTIONAL_DATE_SCHEMA, TimestampConverter.OPTIONAL_TIME_SCHEMA); + testWithSchemaNullValueConversion("Time", Schema.OPTIONAL_STRING_SCHEMA, TimestampConverter.OPTIONAL_TIME_SCHEMA); + testWithSchemaNullValueConversion("Time", TimestampConverter.OPTIONAL_TIMESTAMP_SCHEMA, TimestampConverter.OPTIONAL_TIME_SCHEMA); + } + + @Test + public void testWithSchemaNullFieldToTime() { + testWithSchemaNullFieldConversion("Time", Schema.OPTIONAL_INT64_SCHEMA, TimestampConverter.OPTIONAL_TIME_SCHEMA); + testWithSchemaNullFieldConversion("Time", TimestampConverter.OPTIONAL_TIME_SCHEMA, TimestampConverter.OPTIONAL_TIME_SCHEMA); + testWithSchemaNullFieldConversion("Time", TimestampConverter.OPTIONAL_DATE_SCHEMA, TimestampConverter.OPTIONAL_TIME_SCHEMA); + testWithSchemaNullFieldConversion("Time", Schema.OPTIONAL_STRING_SCHEMA, TimestampConverter.OPTIONAL_TIME_SCHEMA); + testWithSchemaNullFieldConversion("Time", TimestampConverter.OPTIONAL_TIMESTAMP_SCHEMA, TimestampConverter.OPTIONAL_TIME_SCHEMA); + } + + @Test + public void testWithSchemaNullValueToDate() { + testWithSchemaNullValueConversion("Date", Schema.OPTIONAL_INT64_SCHEMA, TimestampConverter.OPTIONAL_DATE_SCHEMA); + testWithSchemaNullValueConversion("Date", TimestampConverter.OPTIONAL_TIME_SCHEMA, TimestampConverter.OPTIONAL_DATE_SCHEMA); + testWithSchemaNullValueConversion("Date", TimestampConverter.OPTIONAL_DATE_SCHEMA, TimestampConverter.OPTIONAL_DATE_SCHEMA); + testWithSchemaNullValueConversion("Date", Schema.OPTIONAL_STRING_SCHEMA, TimestampConverter.OPTIONAL_DATE_SCHEMA); + testWithSchemaNullValueConversion("Date", TimestampConverter.OPTIONAL_TIMESTAMP_SCHEMA, TimestampConverter.OPTIONAL_DATE_SCHEMA); + } + + @Test + public void testWithSchemaNullFieldToDate() { + testWithSchemaNullFieldConversion("Date", Schema.OPTIONAL_INT64_SCHEMA, TimestampConverter.OPTIONAL_DATE_SCHEMA); + testWithSchemaNullFieldConversion("Date", TimestampConverter.OPTIONAL_TIME_SCHEMA, TimestampConverter.OPTIONAL_DATE_SCHEMA); + testWithSchemaNullFieldConversion("Date", TimestampConverter.OPTIONAL_DATE_SCHEMA, TimestampConverter.OPTIONAL_DATE_SCHEMA); + testWithSchemaNullFieldConversion("Date", Schema.OPTIONAL_STRING_SCHEMA, TimestampConverter.OPTIONAL_DATE_SCHEMA); + testWithSchemaNullFieldConversion("Date", TimestampConverter.OPTIONAL_TIMESTAMP_SCHEMA, TimestampConverter.OPTIONAL_DATE_SCHEMA); + } + + @Test + public void testWithSchemaNullValueToString() { + testWithSchemaNullValueConversion("string", Schema.OPTIONAL_INT64_SCHEMA, Schema.OPTIONAL_STRING_SCHEMA); + testWithSchemaNullValueConversion("string", TimestampConverter.OPTIONAL_TIME_SCHEMA, Schema.OPTIONAL_STRING_SCHEMA); + testWithSchemaNullValueConversion("string", TimestampConverter.OPTIONAL_DATE_SCHEMA, Schema.OPTIONAL_STRING_SCHEMA); + testWithSchemaNullValueConversion("string", Schema.OPTIONAL_STRING_SCHEMA, Schema.OPTIONAL_STRING_SCHEMA); + testWithSchemaNullValueConversion("string", TimestampConverter.OPTIONAL_TIMESTAMP_SCHEMA, Schema.OPTIONAL_STRING_SCHEMA); + } + + @Test + public void testWithSchemaNullFieldToString() { + testWithSchemaNullFieldConversion("string", Schema.OPTIONAL_INT64_SCHEMA, Schema.OPTIONAL_STRING_SCHEMA); + testWithSchemaNullFieldConversion("string", TimestampConverter.OPTIONAL_TIME_SCHEMA, Schema.OPTIONAL_STRING_SCHEMA); + testWithSchemaNullFieldConversion("string", TimestampConverter.OPTIONAL_DATE_SCHEMA, Schema.OPTIONAL_STRING_SCHEMA); + testWithSchemaNullFieldConversion("string", Schema.OPTIONAL_STRING_SCHEMA, Schema.OPTIONAL_STRING_SCHEMA); + testWithSchemaNullFieldConversion("string", TimestampConverter.OPTIONAL_TIMESTAMP_SCHEMA, Schema.OPTIONAL_STRING_SCHEMA); + } + + private void testWithSchemaNullValueConversion(String targetType, Schema originalSchema, Schema expectedSchema) { + Map config = new HashMap<>(); + config.put(TimestampConverter.TARGET_TYPE_CONFIG, targetType); + config.put(TimestampConverter.FORMAT_CONFIG, STRING_DATE_FMT); + xformValue.configure(config); + SourceRecord transformed = xformValue.apply(createRecordWithSchema(originalSchema, null)); + + assertEquals(expectedSchema, transformed.valueSchema()); + assertNull(transformed.value()); + } + + private void testWithSchemaNullFieldConversion(String targetType, Schema originalSchema, Schema expectedSchema) { + Map config = new HashMap<>(); + config.put(TimestampConverter.TARGET_TYPE_CONFIG, targetType); + config.put(TimestampConverter.FORMAT_CONFIG, STRING_DATE_FMT); + config.put(TimestampConverter.FIELD_CONFIG, "ts"); + xformValue.configure(config); + SchemaBuilder structSchema = SchemaBuilder.struct() + .field("ts", originalSchema) + .field("other", Schema.STRING_SCHEMA); + + SchemaBuilder expectedStructSchema = SchemaBuilder.struct() + .field("ts", expectedSchema) + .field("other", Schema.STRING_SCHEMA); + + Struct original = new Struct(structSchema); + original.put("ts", null); + original.put("other", "test"); + + // Struct field is null + SourceRecord transformed = xformValue.apply(createRecordWithSchema(structSchema.build(), original)); + + assertEquals(expectedStructSchema.build(), transformed.valueSchema()); + assertNull(requireStruct(transformed.value(), "").get("ts")); + + // entire Struct is null + transformed = xformValue.apply(createRecordWithSchema(structSchema.optional().build(), null)); + + assertEquals(expectedStructSchema.optional().build(), transformed.valueSchema()); + assertNull(transformed.value()); + } // Convert field instead of entire key/value @@ -306,7 +490,7 @@ public void testSchemalessFieldConversion() { xformValue.configure(config); Object value = Collections.singletonMap("ts", DATE_PLUS_TIME.getTime()); - SourceRecord transformed = xformValue.apply(new SourceRecord(null, null, "topic", 0, null, value)); + SourceRecord transformed = xformValue.apply(createRecordSchemaless(value)); assertNull(transformed.valueSchema()); assertEquals(Collections.singletonMap("ts", DATE.getTime()), transformed.value()); @@ -328,7 +512,7 @@ public void testWithSchemaFieldConversion() { original.put("ts", DATE_PLUS_TIME_UNIX); original.put("other", "test"); - SourceRecord transformed = xformValue.apply(new SourceRecord(null, null, "topic", 0, structWithTimestampFieldSchema, original)); + SourceRecord transformed = xformValue.apply(createRecordWithSchema(structWithTimestampFieldSchema, original)); Schema expectedSchema = SchemaBuilder.struct() .field("ts", Timestamp.SCHEMA) @@ -351,4 +535,11 @@ public void testKey() { assertEquals(DATE_PLUS_TIME.getTime(), transformed.key()); } + private SourceRecord createRecordWithSchema(Schema schema, Object value) { + return new SourceRecord(null, null, "topic", 0, schema, value); + } + + private SourceRecord createRecordSchemaless(Object value) { + return createRecordWithSchema(null, value); + } } diff --git a/core/src/main/scala/kafka/admin/AclCommand.scala b/core/src/main/scala/kafka/admin/AclCommand.scala index 25b630d4159b2..6f2384a6967dc 100644 --- a/core/src/main/scala/kafka/admin/AclCommand.scala +++ b/core/src/main/scala/kafka/admin/AclCommand.scala @@ -25,18 +25,18 @@ import kafka.utils._ import org.apache.kafka.common.security.JaasUtils import org.apache.kafka.common.security.auth.KafkaPrincipal import org.apache.kafka.common.utils.Utils -import org.apache.kafka.common.resource.{ResourceFilter, ResourceNameType => JResourceNameType, ResourceType => JResourceType, Resource => JResource} +import org.apache.kafka.common.resource.{PatternType, ResourcePatternFilter, Resource => JResource, ResourceType => JResourceType} import scala.collection.JavaConverters._ object AclCommand extends Logging { - val ClusterResourceFilter = new ResourceFilter(JResourceType.CLUSTER, JResource.CLUSTER_NAME, JResourceNameType.LITERAL) + val ClusterResourceFilter = new ResourcePatternFilter(JResourceType.CLUSTER, JResource.CLUSTER_NAME, PatternType.LITERAL) private val Newline = scala.util.Properties.lineSeparator val ResourceTypeToValidOperations: Map[JResourceType, Set[Operation]] = Map[JResourceType, Set[Operation]]( - JResourceType.TOPIC -> Set(Read, Write, Create, Describe, Delete, DescribeConfigs, AlterConfigs, All), + JResourceType.TOPIC -> Set(Read, Write, Create, Describe, Delete, Alter, DescribeConfigs, AlterConfigs, All), JResourceType.GROUP -> Set(Read, Describe, Delete, All), JResourceType.CLUSTER -> Set(Create, ClusterAction, DescribeConfigs, AlterConfigs, IdempotentWrite, Alter, Describe, All), JResourceType.TRANSACTIONAL_ID -> Set(Describe, Write, All), @@ -87,13 +87,14 @@ object AclCommand extends Logging { } private def addAcl(opts: AclCommandOptions) { - if (opts.options.valueOf(opts.resourceNameType) == JResourceNameType.ANY) - CommandLineUtils.printUsageAndDie(opts.parser, "A '--resource-name-type' value of 'Any' is not valid when adding acls.") + val patternType: PatternType = opts.options.valueOf(opts.resourcePatternType) + if (patternType == PatternType.MATCH || patternType == PatternType.ANY) + CommandLineUtils.printUsageAndDie(opts.parser, s"A '--resource-pattern-type' value of '$patternType' is not valid when adding acls.") withAuthorizer(opts) { authorizer => val resourceToAcl = getResourceFilterToAcls(opts).map { case (filter, acls) => - Resource(ResourceType.fromJava(filter.resourceType()), filter.name(), ResourceNameType.fromJava(filter.nameType())) -> acls + Resource(ResourceType.fromJava(filter.resourceType()), filter.name(), filter.patternType()) -> acls } if (resourceToAcl.values.exists(_.isEmpty)) @@ -126,7 +127,7 @@ object AclCommand extends Logging { } } - private def removeAcls(authorizer: Authorizer, acls: Set[Acl], filter: ResourceFilter) { + private def removeAcls(authorizer: Authorizer, acls: Set[Acl], filter: ResourcePatternFilter) { getAcls(authorizer, filter) .keys .foreach(resource => @@ -148,12 +149,12 @@ object AclCommand extends Logging { } } - private def getAcls(authorizer: Authorizer, filter: ResourceFilter): Map[Resource, Set[Acl]] = + private def getAcls(authorizer: Authorizer, filter: ResourcePatternFilter): Map[Resource, Set[Acl]] = authorizer.getAcls() - .filter { case (resource, acl) => filter.matches(resource.toJava) } + .filter { case (resource, acl) => filter.matches(resource.toPattern) } - private def getResourceFilterToAcls(opts: AclCommandOptions): Map[ResourceFilter, Set[Acl]] = { - var resourceToAcls = Map.empty[ResourceFilter, Set[Acl]] + private def getResourceFilterToAcls(opts: AclCommandOptions): Map[ResourcePatternFilter, Set[Acl]] = { + var resourceToAcls = Map.empty[ResourcePatternFilter, 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)) { @@ -172,11 +173,11 @@ object AclCommand extends Logging { resourceToAcls } - private def getProducerResourceFilterToAcls(opts: AclCommandOptions): Map[ResourceFilter, Set[Acl]] = { + private def getProducerResourceFilterToAcls(opts: AclCommandOptions): Map[ResourcePatternFilter, Set[Acl]] = { val filters = getResourceFilter(opts) - val topics: Set[ResourceFilter] = filters.filter(_.resourceType == JResourceType.TOPIC) - val transactionalIds: Set[ResourceFilter] = filters.filter(_.resourceType == JResourceType.TRANSACTIONAL_ID) + val topics: Set[ResourcePatternFilter] = filters.filter(_.resourceType == JResourceType.TOPIC) + val transactionalIds: Set[ResourcePatternFilter] = filters.filter(_.resourceType == JResourceType.TRANSACTIONAL_ID) val enableIdempotence = opts.options.has(opts.idempotentOpt) val topicAcls = getAcl(opts, Set(Write, Describe, Create)) @@ -185,27 +186,27 @@ object AclCommand extends Logging { //Write, Describe, Create permission on topics, Write, Describe on transactionalIds topics.map(_ -> topicAcls).toMap ++ transactionalIds.map(_ -> transactionalIdAcls).toMap ++ - (if (enableIdempotence) + (if (enableIdempotence) Map(ClusterResourceFilter -> getAcl(opts, Set(IdempotentWrite))) else Map.empty) } - private def getConsumerResourceFilterToAcls(opts: AclCommandOptions): Map[ResourceFilter, Set[Acl]] = { + private def getConsumerResourceFilterToAcls(opts: AclCommandOptions): Map[ResourcePatternFilter, Set[Acl]] = { val filters = getResourceFilter(opts) - val topics: Set[ResourceFilter] = filters.filter(_.resourceType == JResourceType.TOPIC) - val groups: Set[ResourceFilter] = filters.filter(_.resourceType == JResourceType.GROUP) + val topics: Set[ResourcePatternFilter] = filters.filter(_.resourceType == JResourceType.TOPIC) + val groups: Set[ResourcePatternFilter] = filters.filter(_.resourceType == JResourceType.GROUP) //Read, Describe on topic, Read on consumerGroup val acls = getAcl(opts, Set(Read, Describe)) - topics.map(_ -> acls).toMap[ResourceFilter, Set[Acl]] ++ - groups.map(_ -> getAcl(opts, Set(Read))).toMap[ResourceFilter, Set[Acl]] + topics.map(_ -> acls).toMap[ResourcePatternFilter, Set[Acl]] ++ + groups.map(_ -> getAcl(opts, Set(Read))).toMap[ResourcePatternFilter, Set[Acl]] } - private def getCliResourceFilterToAcls(opts: AclCommandOptions): Map[ResourceFilter, Set[Acl]] = { + private def getCliResourceFilterToAcls(opts: AclCommandOptions): Map[ResourcePatternFilter, Set[Acl]] = { val acls = getAcl(opts) val filters = getResourceFilter(opts) filters.map(_ -> acls).toMap @@ -261,25 +262,25 @@ object AclCommand extends Logging { Set.empty[KafkaPrincipal] } - private def getResourceFilter(opts: AclCommandOptions, dieIfNoResourceFound: Boolean = true): Set[ResourceFilter] = { - val resourceNameType: JResourceNameType = opts.options.valueOf(opts.resourceNameType) + private def getResourceFilter(opts: AclCommandOptions, dieIfNoResourceFound: Boolean = true): Set[ResourcePatternFilter] = { + val patternType: PatternType = opts.options.valueOf(opts.resourcePatternType) - var resourceFilters = Set.empty[ResourceFilter] + var resourceFilters = Set.empty[ResourcePatternFilter] if (opts.options.has(opts.topicOpt)) - opts.options.valuesOf(opts.topicOpt).asScala.foreach(topic => resourceFilters += new ResourceFilter(JResourceType.TOPIC, topic.trim, resourceNameType)) + opts.options.valuesOf(opts.topicOpt).asScala.foreach(topic => resourceFilters += new ResourcePatternFilter(JResourceType.TOPIC, topic.trim, patternType)) - if (resourceNameType == JResourceNameType.LITERAL && (opts.options.has(opts.clusterOpt) || opts.options.has(opts.idempotentOpt))) + if (patternType == PatternType.LITERAL && (opts.options.has(opts.clusterOpt) || opts.options.has(opts.idempotentOpt))) resourceFilters += ClusterResourceFilter if (opts.options.has(opts.groupOpt)) - opts.options.valuesOf(opts.groupOpt).asScala.foreach(group => resourceFilters += new ResourceFilter(JResourceType.GROUP, group.trim, resourceNameType)) + opts.options.valuesOf(opts.groupOpt).asScala.foreach(group => resourceFilters += new ResourcePatternFilter(JResourceType.GROUP, group.trim, patternType)) if (opts.options.has(opts.transactionalIdOpt)) opts.options.valuesOf(opts.transactionalIdOpt).asScala.foreach(transactionalId => - resourceFilters += new ResourceFilter(JResourceType.TRANSACTIONAL_ID, transactionalId, resourceNameType)) + resourceFilters += new ResourcePatternFilter(JResourceType.TRANSACTIONAL_ID, transactionalId, patternType)) if (opts.options.has(opts.delegationTokenOpt)) - opts.options.valuesOf(opts.delegationTokenOpt).asScala.foreach(token => resourceFilters += new ResourceFilter(JResourceType.DELEGATION_TOKEN, token.trim, resourceNameType)) + opts.options.valuesOf(opts.delegationTokenOpt).asScala.foreach(token => resourceFilters += new ResourcePatternFilter(JResourceType.DELEGATION_TOKEN, token.trim, patternType)) if (resourceFilters.isEmpty && dieIfNoResourceFound) CommandLineUtils.printUsageAndDie(opts.parser, "You must provide at least one resource: --topic or --cluster or --group or --delegation-token ") @@ -294,7 +295,7 @@ object AclCommand extends Logging { Console.readLine().equalsIgnoreCase("y") } - private def validateOperation(opts: AclCommandOptions, resourceToAcls: Map[ResourceFilter, Set[Acl]]): Unit = { + private def validateOperation(opts: AclCommandOptions, resourceToAcls: Map[ResourcePatternFilter, Set[Acl]]): Unit = { for ((resource, acls) <- resourceToAcls) { val validOps = ResourceTypeToValidOperations(resource.resourceType) if ((acls.map(_.operation) -- validOps).nonEmpty) @@ -345,11 +346,16 @@ object AclCommand extends Logging { .describedAs("delegation-token") .ofType(classOf[String]) - val resourceNameType = parser.accepts("resource-name-type", "The type of the resource name, or any.") + val resourcePatternType = parser.accepts("resource-pattern-type", "The type of the resource pattern or pattern filter. " + + "When adding acls, this should be a specific pattern type, e.g. 'literal' or 'prefixed'. " + + "When listing or removing acls, a specific pattern type can be used to list or remove acls from specific resource patterns, " + + "or use the filter values of 'any' or 'match', where 'any' will match any pattern type, but will match the resource name exactly, " + + "where as 'match' will perform pattern matching to list or remove all acls that affect the supplied resource(s). " + + "WARNING: 'match', when used in combination with the '--remove' switch, should be used with care.") .withRequiredArg() .ofType(classOf[String]) - .withValuesConvertedBy(new ResourceNameTypeConverter()) - .defaultsTo(JResourceNameType.LITERAL) + .withValuesConvertedBy(new PatternTypeConverter()) + .defaultsTo(PatternType.LITERAL) val addOpt = parser.accepts("add", "Indicates you are trying to add ACLs.") val removeOpt = parser.accepts("remove", "Indicates you are trying to remove ACLs.") @@ -429,17 +435,17 @@ object AclCommand extends Logging { } -class ResourceNameTypeConverter extends EnumConverter[JResourceNameType](classOf[JResourceNameType]) { +class PatternTypeConverter extends EnumConverter[PatternType](classOf[PatternType]) { - override def convert(value: String): JResourceNameType = { - val nameType = super.convert(value) - if (nameType.isUnknown) - throw new ValueConversionException("Unknown resourceNameType: " + value) + override def convert(value: String): PatternType = { + val patternType = super.convert(value) + if (patternType.isUnknown) + throw new ValueConversionException("Unknown resource-pattern-type: " + value) - nameType + patternType } - override def valuePattern: String = JResourceNameType.values - .filter(_ != JResourceNameType.UNKNOWN) + override def valuePattern: String = PatternType.values + .filter(_ != PatternType.UNKNOWN) .mkString("|") } diff --git a/core/src/main/scala/kafka/admin/AdminClient.scala b/core/src/main/scala/kafka/admin/AdminClient.scala index ea42530e66d2b..1009bc5d278af 100644 --- a/core/src/main/scala/kafka/admin/AdminClient.scala +++ b/core/src/main/scala/kafka/admin/AdminClient.scala @@ -22,7 +22,7 @@ import kafka.common.KafkaException import kafka.coordinator.group.GroupOverview import kafka.utils.Logging import org.apache.kafka.clients._ -import org.apache.kafka.clients.consumer.internals.{ConsumerNetworkClient, ConsumerProtocol, RequestFuture, RequestFutureAdapter} +import org.apache.kafka.clients.consumer.internals.{ConsumerNetworkClient, ConsumerProtocol, RequestFuture} import org.apache.kafka.common.config.ConfigDef.{Importance, Type} import org.apache.kafka.common.config.{AbstractConfig, ConfigDef} import org.apache.kafka.common.errors.{AuthenticationException, TimeoutException} @@ -42,9 +42,10 @@ import scala.util.{Failure, Success, Try} /** * A Scala administrative client for Kafka which supports managing and inspecting topics, brokers, - * and configurations. This client is deprecated, and will be replaced by KafkaAdminClient. - * @see KafkaAdminClient + * and configurations. This client is deprecated, and will be replaced by org.apache.kafka.clients.admin.AdminClient. */ +@deprecated("This class is deprecated in favour of org.apache.kafka.clients.admin.AdminClient and it will be removed in " + + "a future release.", since = "0.11.0") class AdminClient(val time: Time, val requestTimeoutMs: Int, val retryBackoffMs: Long, @@ -364,6 +365,8 @@ class CompositeFuture[T](time: Time, } } +@deprecated("This class is deprecated in favour of org.apache.kafka.clients.admin.AdminClient and it will be removed in " + + "a future release.", since = "0.11.0") object AdminClient { val DefaultConnectionMaxIdleMs = 9 * 60 * 1000 val DefaultRequestTimeoutMs = 5000 @@ -460,7 +463,7 @@ object AdminClient { metadata, time, retryBackoffMs, - requestTimeoutMs.toLong, + requestTimeoutMs, Integer.MAX_VALUE) new AdminClient( diff --git a/core/src/main/scala/kafka/admin/AdminUtils.scala b/core/src/main/scala/kafka/admin/AdminUtils.scala index 2ae03aa65f209..4dae6eb25e43b 100644 --- a/core/src/main/scala/kafka/admin/AdminUtils.scala +++ b/core/src/main/scala/kafka/admin/AdminUtils.scala @@ -363,76 +363,17 @@ object AdminUtils extends Logging with AdminUtilities { @deprecated("This method is deprecated and will be replaced by kafka.zk.AdminZkClient.", "1.1.0") def deleteTopic(zkUtils: ZkUtils, topic: String) { - if (topicExists(zkUtils, topic)) { - try { - zkUtils.createPersistentPath(getDeleteTopicPath(topic)) - } catch { - case _: ZkNodeExistsException => throw new TopicAlreadyMarkedForDeletionException( - "topic %s is already marked for deletion".format(topic)) - case e2: Throwable => throw new AdminOperationException(e2) - } - } else { - throw new UnknownTopicOrPartitionException(s"Topic `$topic` to delete does not exist") + if (topicExists(zkUtils, topic)) { + try { + zkUtils.createPersistentPath(getDeleteTopicPath(topic)) + } catch { + case _: ZkNodeExistsException => throw new TopicAlreadyMarkedForDeletionException( + "topic %s is already marked for deletion".format(topic)) + case e2: Throwable => throw new AdminOperationException(e2) } + } else { + throw new UnknownTopicOrPartitionException(s"Topic `$topic` to delete does not exist") } - - @deprecated("This method has been deprecated and will be removed in a future release.", "0.11.0.0") - def isConsumerGroupActive(zkUtils: ZkUtils, group: String) = { - zkUtils.getConsumersInGroup(group).nonEmpty - } - - /** - * Delete the whole directory of the given consumer group if the group is inactive. - * - * @param zkUtils Zookeeper utilities - * @param group Consumer group - * @return whether or not we deleted the consumer group information - */ - @deprecated("This method has been deprecated and will be removed in a future release.", "0.11.0.0") - def deleteConsumerGroupInZK(zkUtils: ZkUtils, group: String) = { - if (!isConsumerGroupActive(zkUtils, group)) { - val dir = new ZKGroupDirs(group) - zkUtils.deletePathRecursive(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 zkUtils Zookeeper utilities - * @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 - */ - @deprecated("This method has been deprecated and will be removed in a future release.", "0.11.0.0") - def deleteConsumerGroupInfoForTopicInZK(zkUtils: ZkUtils, group: String, topic: String) = { - val topics = zkUtils.getTopicsByConsumerGroup(group) - if (topics == Seq(topic)) { - deleteConsumerGroupInZK(zkUtils, group) - } - else if (!isConsumerGroupActive(zkUtils, group)) { - val dir = new ZKGroupTopicDirs(group, topic) - zkUtils.deletePathRecursive(dir.consumerOwnerDir) - zkUtils.deletePathRecursive(dir.consumerOffsetDir) - true - } - else false - } - - /** - * Delete every inactive consumer group's information about the given topic in Zookeeper. - * - * @param zkUtils Zookeeper utilities - * @param topic Topic of the consumer group information we wish to delete - */ - @deprecated("This method has been deprecated and will be removed in a future release.", "0.11.0.0") - def deleteAllConsumerGroupInfoForTopicInZK(zkUtils: ZkUtils, topic: String): Set[String] = { - val groups = zkUtils.getAllConsumerGroupsForTopic(topic) - groups.foreach(group => deleteConsumerGroupInfoForTopicInZK(zkUtils, group, topic)) - groups } def topicExists(zkUtils: ZkUtils, topic: String): Boolean = diff --git a/core/src/main/scala/kafka/admin/ConfigCommand.scala b/core/src/main/scala/kafka/admin/ConfigCommand.scala index c1b384fd1fa5d..d8dade06e7b86 100644 --- a/core/src/main/scala/kafka/admin/ConfigCommand.scala +++ b/core/src/main/scala/kafka/admin/ConfigCommand.scala @@ -22,15 +22,16 @@ import java.util.{Collections, Properties} import joptsimple._ import kafka.common.Config -import kafka.common.InvalidConfigException import kafka.log.LogConfig -import kafka.server.{ConfigEntityName, ConfigType, DynamicConfig} -import kafka.utils.{CommandLineUtils, Exit} +import kafka.server.{ConfigEntityName, ConfigType, Defaults, DynamicBrokerConfig, DynamicConfig, KafkaConfig} +import kafka.utils.{CommandLineUtils, Exit, PasswordEncoder} import kafka.utils.Implicits._ import kafka.zk.{AdminZkClient, KafkaZkClient} import org.apache.kafka.clients.CommonClientConfigs import org.apache.kafka.clients.admin.{AlterConfigsOptions, ConfigEntry, DescribeConfigsOptions, AdminClient => JAdminClient, Config => JConfig} import org.apache.kafka.common.config.ConfigResource +import org.apache.kafka.common.config.types.Password +import org.apache.kafka.common.errors.InvalidConfigurationException import org.apache.kafka.common.security.JaasUtils import org.apache.kafka.common.security.scram.internals.{ScramCredentialUtils, ScramFormatter, ScramMechanism} import org.apache.kafka.common.utils.{Sanitizer, Time, Utils} @@ -56,11 +57,14 @@ import scala.collection.JavaConverters._ object ConfigCommand extends Config { val DefaultScramIterations = 4096 - // Dynamic broker configs can only be updated using the new AdminClient since they may require - // password encryption currently implemented only in the broker. For consistency with older versions, - // quota-related broker configs can still be updated using ZooKeeper. ConfigCommand will be migrated - // fully to the new AdminClient later (KIP-248). - val BrokerConfigsUpdatableUsingZooKeeper = Set(DynamicConfig.Broker.LeaderReplicationThrottledRateProp, + // Dynamic broker configs can only be updated using the new AdminClient once brokers have started + // so that configs may be fully validated. Prior to starting brokers, updates may be performed using + // ZooKeeper for bootstrapping. This allows all password configs to be stored encrypted in ZK, + // avoiding clear passwords in server.properties. For consistency with older versions, quota-related + // broker configs can still be updated using ZooKeeper at any time. ConfigCommand will be migrated + // to the new AdminClient later for these configs (KIP-248). + val BrokerConfigsUpdatableUsingZooKeeperWhileBrokerRunning = Set( + DynamicConfig.Broker.LeaderReplicationThrottledRateProp, DynamicConfig.Broker.FollowerReplicationThrottledRateProp, DynamicConfig.Broker.ReplicaAlterLogDirsIoMaxBytesPerSecondProp) @@ -79,7 +83,7 @@ object ConfigCommand extends Config { processBrokerConfig(opts) } } catch { - case e @ (_: IllegalArgumentException | _: InvalidConfigException | _: OptionException) => + case e @ (_: IllegalArgumentException | _: InvalidConfigurationException | _: OptionException) => logger.debug(s"Failed config command with args '${args.mkString(" ")}'", e) System.err.println(e.getMessage) Exit.exit(1) @@ -114,9 +118,25 @@ object ConfigCommand extends Config { if (entityType == ConfigType.User) preProcessScramCredentials(configsToBeAdded) - if (entityType == ConfigType.Broker) { - require(configsToBeAdded.asScala.keySet.forall(BrokerConfigsUpdatableUsingZooKeeper.contains), - s"--bootstrap-server option must be specified to update broker configs $configsToBeAdded") + else if (entityType == ConfigType.Broker) { + // Replication quota configs may be updated using ZK at any time. Other dynamic broker configs + // may be updated using ZooKeeper only if the corresponding broker is not running. Dynamic broker + // configs at cluster-default level may be configured using ZK only if there are no brokers running. + val dynamicBrokerConfigs = configsToBeAdded.asScala.keySet.filterNot(BrokerConfigsUpdatableUsingZooKeeperWhileBrokerRunning.contains) + if (dynamicBrokerConfigs.nonEmpty) { + val perBrokerConfig = entityName != ConfigEntityName.Default + val errorMessage = s"--bootstrap-server option must be specified to update broker configs $dynamicBrokerConfigs." + val info = "Broker configuraton updates using ZooKeeper are supported for bootstrapping before brokers" + + " are started to enable encrypted password configs to be stored in ZooKeeper." + if (perBrokerConfig) { + adminZkClient.parseBroker(entityName).foreach { brokerId => + require(zkClient.getBroker(brokerId).isEmpty, s"$errorMessage when broker $entityName is running. $info") + } + } else { + require(zkClient.getAllBrokersInCluster.isEmpty, s"$errorMessage for default cluster if any broker is running. $info") + } + preProcessBrokerConfigs(configsToBeAdded, perBrokerConfig) + } } // compile the final set of configs @@ -125,7 +145,7 @@ object ConfigCommand extends Config { // fail the command if any of the configs to be deleted does not exist val invalidConfigs = configsToBeDeleted.filterNot(configs.containsKey(_)) if (invalidConfigs.nonEmpty) - throw new InvalidConfigException(s"Invalid config(s): ${invalidConfigs.mkString(",")}") + throw new InvalidConfigurationException(s"Invalid config(s): ${invalidConfigs.mkString(",")}") configs ++= configsToBeAdded configsToBeDeleted.foreach(configs.remove(_)) @@ -156,6 +176,49 @@ object ConfigCommand extends Config { } } + private[admin] def createPasswordEncoder(encoderConfigs: Map[String, String]): PasswordEncoder = { + encoderConfigs.get(KafkaConfig.PasswordEncoderSecretProp) + val encoderSecret = encoderConfigs.getOrElse(KafkaConfig.PasswordEncoderSecretProp, + throw new IllegalArgumentException("Password encoder secret not specified")) + new PasswordEncoder(new Password(encoderSecret), + None, + encoderConfigs.get(KafkaConfig.PasswordEncoderCipherAlgorithmProp).getOrElse(Defaults.PasswordEncoderCipherAlgorithm), + encoderConfigs.get(KafkaConfig.PasswordEncoderKeyLengthProp).map(_.toInt).getOrElse(Defaults.PasswordEncoderKeyLength), + encoderConfigs.get(KafkaConfig.PasswordEncoderIterationsProp).map(_.toInt).getOrElse(Defaults.PasswordEncoderIterations)) + } + + /** + * Pre-process broker configs provided to convert them to persistent format. + * Password configs are encrypted using the secret `KafkaConfig.PasswordEncoderSecretProp`. + * The secret is removed from `configsToBeAdded` and will not be persisted in ZooKeeper. + */ + private def preProcessBrokerConfigs(configsToBeAdded: Properties, perBrokerConfig: Boolean) { + val passwordEncoderConfigs = new Properties + passwordEncoderConfigs ++= configsToBeAdded.asScala.filterKeys(_.startsWith("password.encoder.")) + if (!passwordEncoderConfigs.isEmpty) { + info(s"Password encoder configs ${passwordEncoderConfigs.keySet} will be used for encrypting" + + " passwords, but will not be stored in ZooKeeper.") + passwordEncoderConfigs.asScala.keySet.foreach(configsToBeAdded.remove) + } + + DynamicBrokerConfig.validateConfigs(configsToBeAdded, perBrokerConfig) + val passwordConfigs = configsToBeAdded.asScala.keySet.filter(DynamicBrokerConfig.isPasswordConfig) + if (passwordConfigs.nonEmpty) { + require(passwordEncoderConfigs.containsKey(KafkaConfig.PasswordEncoderSecretProp), + s"${KafkaConfig.PasswordEncoderSecretProp} must be specified to update $passwordConfigs." + + " Other password encoder configs like cipher algorithm and iterations may also be specified" + + " to override the default encoding parameters. Password encoder configs will not be persisted" + + " in ZooKeeper." + ) + + val passwordEncoder = createPasswordEncoder(passwordEncoderConfigs.asScala) + passwordConfigs.foreach { configName => + val encodedValue = passwordEncoder.encode(new Password(configsToBeAdded.getProperty(configName))) + configsToBeAdded.setProperty(configName, encodedValue) + } + } + } + private def describeConfig(zkClient: KafkaZkClient, opts: ConfigCommandOptions, adminZkClient: AdminZkClient) { val configEntity = parseEntity(opts) val describeAllUsers = configEntity.root.entityType == ConfigType.User && !configEntity.root.sanitizedName.isDefined && !configEntity.child.isDefined @@ -244,12 +307,12 @@ object ConfigCommand extends Config { // fail the command if any of the configs to be deleted does not exist val invalidConfigs = configsToBeDeleted.filterNot(oldConfig.contains) if (invalidConfigs.nonEmpty) - throw new InvalidConfigException(s"Invalid config(s): ${invalidConfigs.mkString(",")}") + throw new InvalidConfigurationException(s"Invalid config(s): ${invalidConfigs.mkString(",")}") val newEntries = oldConfig ++ configsToBeAdded -- configsToBeDeleted val sensitiveEntries = newEntries.filter(_._2.value == null) if (sensitiveEntries.nonEmpty) - throw new InvalidConfigException(s"All sensitive broker config entries must be specified for --alter, missing entries: ${sensitiveEntries.keySet}") + throw new InvalidConfigurationException(s"All sensitive broker config entries must be specified for --alter, missing entries: ${sensitiveEntries.keySet}") val newConfig = new JConfig(newEntries.asJava.values) val alterOptions = new AlterConfigsOptions().timeoutMs(30000).validateOnly(false) @@ -358,7 +421,12 @@ object ConfigCommand extends Config { parseQuotaEntity(opts) else { // Exactly one entity type and at-most one entity name expected for other entities - val name = if (opts.options.has(opts.entityName)) Some(opts.options.valueOf(opts.entityName)) else None + val name = if (opts.options.has(opts.entityName)) + Some(opts.options.valueOf(opts.entityName)) + else if (entityTypes.head == ConfigType.Broker && opts.options.has(opts.entityDefault)) + Some(ConfigEntityName.Default) + else + None ConfigEntity(Entity(entityTypes.head, name), None) } } diff --git a/core/src/main/scala/kafka/admin/ConsumerGroupCommand.scala b/core/src/main/scala/kafka/admin/ConsumerGroupCommand.scala index 44d09fda5fa6e..2b5da4ff2b110 100755 --- a/core/src/main/scala/kafka/admin/ConsumerGroupCommand.scala +++ b/core/src/main/scala/kafka/admin/ConsumerGroupCommand.scala @@ -22,28 +22,19 @@ import java.util.{Date, Properties} import javax.xml.datatype.DatatypeFactory import joptsimple.{OptionParser, OptionSpec} -import kafka.api.{OffsetFetchRequest, OffsetFetchResponse, OffsetRequest, PartitionOffsetRequestInfo} -import kafka.client.ClientUtils -import kafka.common.{OffsetMetadataAndError, TopicAndPartition} -import kafka.consumer.SimpleConsumer + import kafka.utils._ import kafka.utils.Implicits._ -import org.I0Itec.zkclient.exception.ZkNoNodeException import org.apache.kafka.clients.CommonClientConfigs import org.apache.kafka.clients.consumer.{ConsumerConfig, KafkaConsumer, OffsetAndMetadata} -import org.apache.kafka.common.errors.BrokerNotAvailableException import org.apache.kafka.common.{KafkaException, Node, TopicPartition} -import org.apache.kafka.common.internals.Topic -import org.apache.kafka.common.network.ListenerName import org.apache.kafka.common.protocol.Errors -import org.apache.kafka.common.security.JaasUtils -import org.apache.kafka.common.security.auth.SecurityProtocol import org.apache.kafka.common.serialization.StringDeserializer import org.apache.kafka.common.utils.Utils import scala.collection.JavaConverters._ -import scala.collection.{Seq, Set, mutable} +import scala.collection.{Seq, Set} object ConsumerGroupCommand extends Logging { @@ -60,15 +51,7 @@ object ConsumerGroupCommand extends Logging { opts.checkArgs() - val consumerGroupService = { - if (opts.useOldConsumer) { - Console.err.println("Note: This will only show information about consumers that use ZooKeeper (not those using the Java consumer API).") - new ZkConsumerGroupService(opts) - } else { - Console.err.println("Note: This will not show information about old Zookeeper-based consumers.") - new KafkaConsumerGroupService(opts) - } - } + val consumerGroupService = new ConsumerGroupService(opts) try { if (opts.options.has(opts.listOpt)) @@ -125,44 +108,47 @@ object ConsumerGroupCommand extends Logging { } } - protected case class PartitionAssignmentState(group: String, coordinator: Option[Node], topic: Option[String], + private[admin] case class PartitionAssignmentState(group: String, coordinator: Option[Node], topic: Option[String], partition: Option[Int], offset: Option[Long], lag: Option[Long], consumerId: Option[String], host: Option[String], clientId: Option[String], logEndOffset: Option[Long]) - protected case class MemberAssignmentState(group: String, consumerId: String, host: String, clientId: String, + private[admin] case class MemberAssignmentState(group: String, consumerId: String, host: String, clientId: String, numPartitions: Int, assignment: List[TopicPartition]) - protected case class GroupState(group: String, coordinator: Node, assignmentStrategy: String, state: String, numMembers: Int) + private[admin] case class GroupState(group: String, coordinator: Node, assignmentStrategy: String, state: String, numMembers: Int) + + class ConsumerGroupService(val opts: ConsumerGroupCommandOptions) { + + private val adminClient = createAdminClient() - sealed trait ConsumerGroupService { + // `consumer` is only needed for `describe`, so we instantiate it lazily + private var consumer: KafkaConsumer[String, String] = _ - def listGroups(): List[String] + def listGroups(): List[String] = { + adminClient.listAllConsumerGroupsFlattened().map(_.groupId) + } private def shouldPrintMemberState(group: String, state: Option[String], numRows: Option[Int]): Boolean = { // numRows contains the number of data rows, if any, compiled from the API call in the caller method. // if it's undefined or 0, there is no relevant group information to display. numRows match { case None => - // applies to both old and new consumer printError(s"The consumer group '$group' does not exist.") false - case Some(num) => - opts.useOldConsumer || { - state match { - case Some("Dead") => - printError(s"Consumer group '$group' does not exist.") - case Some("Empty") => - Console.err.println(s"Consumer group '$group' has no active members.") - case Some("PreparingRebalance") | Some("CompletingRebalance") => - Console.err.println(s"Warning: Consumer group '$group' is rebalancing.") - case Some("Stable") => - case other => - // the control should never reach here - throw new KafkaException(s"Expected a valid consumer group state, but found '${other.getOrElse("NONE")}'.") - } - !state.contains("Dead") && num > 0 - } + case Some(num) => state match { + case Some("Dead") => + printError(s"Consumer group '$group' does not exist.") + case Some("Empty") => + Console.err.println(s"Consumer group '$group' has no active members.") + case Some("PreparingRebalance") | Some("CompletingRebalance") => + Console.err.println(s"Warning: Consumer group '$group' is rebalancing.") + case Some("Stable") => + case other => + // the control should never reach here + throw new KafkaException(s"Expected a valid consumer group state, but found '${other.getOrElse("NONE")}'.") + } + !state.contains("Dead") && num > 0 } } @@ -178,30 +164,22 @@ object ConsumerGroupCommand extends Logging { consumerAssignments.foreach { consumerAssignment => maxTopicLen = Math.max(maxTopicLen, consumerAssignment.topic.getOrElse(MISSING_COLUMN_VALUE).length) maxConsumerIdLen = Math.max(maxConsumerIdLen, consumerAssignment.consumerId.getOrElse(MISSING_COLUMN_VALUE).length) - if (!opts.useOldConsumer) - maxHostLen = Math.max(maxHostLen, consumerAssignment.host.getOrElse(MISSING_COLUMN_VALUE).length) + maxHostLen = Math.max(maxHostLen, consumerAssignment.host.getOrElse(MISSING_COLUMN_VALUE).length) } } - print(s"\n%${-maxTopicLen}s %-10s %-15s %-15s %-15s %${-maxConsumerIdLen}s " - .format("TOPIC", "PARTITION", "CURRENT-OFFSET", "LOG-END-OFFSET", "LAG", "CONSUMER-ID")) - - if (!opts.useOldConsumer) - print(s"%${-maxHostLen}s %s".format("HOST", "CLIENT-ID")) - println() + println(s"\n%${-maxTopicLen}s %-10s %-15s %-15s %-15s %${-maxConsumerIdLen}s %${-maxHostLen}s %s" + .format("TOPIC", "PARTITION", "CURRENT-OFFSET", "LOG-END-OFFSET", "LAG", "CONSUMER-ID", "HOST", "CLIENT-ID")) assignments match { case None => // do nothing case Some(consumerAssignments) => consumerAssignments.foreach { consumerAssignment => - print(s"%${-maxTopicLen}s %-10s %-15s %-15s %-15s %${-maxConsumerIdLen}s ".format( + println(s"%-${maxTopicLen}s %-10s %-15s %-15s %-15s %-${maxConsumerIdLen}s %-${maxHostLen}s %s".format( consumerAssignment.topic.getOrElse(MISSING_COLUMN_VALUE), consumerAssignment.partition.getOrElse(MISSING_COLUMN_VALUE), consumerAssignment.offset.getOrElse(MISSING_COLUMN_VALUE), consumerAssignment.logEndOffset.getOrElse(MISSING_COLUMN_VALUE), - consumerAssignment.lag.getOrElse(MISSING_COLUMN_VALUE), consumerAssignment.consumerId.getOrElse(MISSING_COLUMN_VALUE))) - if (!opts.useOldConsumer) - print(s"%${-maxHostLen}s %s".format(consumerAssignment.host.getOrElse(MISSING_COLUMN_VALUE), - consumerAssignment.clientId.getOrElse(MISSING_COLUMN_VALUE))) - println() + consumerAssignment.lag.getOrElse(MISSING_COLUMN_VALUE), consumerAssignment.consumerId.getOrElse(MISSING_COLUMN_VALUE), + consumerAssignment.host.getOrElse(MISSING_COLUMN_VALUE), consumerAssignment.clientId.getOrElse(MISSING_COLUMN_VALUE))) } } } @@ -250,7 +228,6 @@ object ConsumerGroupCommand extends Logging { } private def printState(group: String, state: GroupState): Unit = { - // this method is reachable only for the new consumer option (where the given state is always defined) if (shouldPrintMemberState(group, Some(state.state), Some(1))) { val coordinator = s"${state.coordinator.host}:${state.coordinator.port} (${state.coordinator.idString})" val coordinatorColLen = Math.max(25, coordinator.length) @@ -277,22 +254,7 @@ object ConsumerGroupCommand extends Logging { printState(group, collectGroupState()) } - def close(): Unit - - protected def opts: ConsumerGroupCommandOptions - - protected def getLogEndOffset(topicPartition: TopicPartition): LogOffsetResult = - getLogEndOffsets(Seq(topicPartition)).getOrElse(topicPartition, LogOffsetResult.Ignore) - - protected def getLogEndOffsets(topicPartitions: Seq[TopicPartition]): Map[TopicPartition, LogOffsetResult] - - def collectGroupOffsets(): (Option[String], Option[Seq[PartitionAssignmentState]]) - - def collectGroupMembers(verbose: Boolean): (Option[String], Option[Seq[MemberAssignmentState]]) = throw new UnsupportedOperationException - - def collectGroupState(): GroupState = throw new UnsupportedOperationException - - protected def collectConsumerAssignment(group: String, + private def collectConsumerAssignment(group: String, coordinator: Option[Node], topicPartitions: Seq[TopicPartition], getPartitionOffset: TopicPartition => Option[Long], @@ -326,233 +288,12 @@ object ConsumerGroupCommand extends Logging { } getLogEndOffsets(topicPartitions).map { - logEndOffsetResult => - logEndOffsetResult._2 match { - case LogOffsetResult.LogOffset(logEndOffset) => getDescribePartitionResult(logEndOffsetResult._1, Some(logEndOffset)) - case LogOffsetResult.Unknown => getDescribePartitionResult(logEndOffsetResult._1, None) - case LogOffsetResult.Ignore => null - } + case (topicPartition, LogOffsetResult.LogOffset(offset)) => getDescribePartitionResult(topicPartition, Some(offset)) + case (topicPartition, _) => getDescribePartitionResult(topicPartition, None) }.toArray } - def resetOffsets(): Map[TopicPartition, OffsetAndMetadata] = throw new UnsupportedOperationException - - def exportOffsetsToReset(assignmentsToReset: Map[TopicPartition, OffsetAndMetadata]): String = throw new UnsupportedOperationException - - def deleteGroups(): Map[String, Errors] - } - - @deprecated("This class has been deprecated and will be removed in a future release.", "0.11.0.0") - class ZkConsumerGroupService(val opts: ConsumerGroupCommandOptions) extends ConsumerGroupService { - - private val zkUtils = { - val zkUrl = opts.options.valueOf(opts.zkConnectOpt) - ZkUtils(zkUrl, 30000, 30000, JaasUtils.isZkSecurityEnabled) - } - - def close() { - zkUtils.close() - } - - def listGroups(): List[String] = { - zkUtils.getConsumerGroups().toList - } - - def deleteGroups(): Map[String, Errors] = { - if (opts.options.has(opts.groupOpt) && opts.options.has(opts.topicOpt)) - deleteGroupsInfoForTopic() - else if (opts.options.has(opts.groupOpt)) - deleteGroupsInfo() - else if (opts.options.has(opts.topicOpt)) - deleteAllGroupsInfoForTopic() - - Map() - } - - def collectGroupOffsets(): (Option[String], Option[Seq[PartitionAssignmentState]]) = { - val group = opts.options.valueOf(opts.groupOpt) - val props = if (opts.options.has(opts.commandConfigOpt)) Utils.loadProps(opts.options.valueOf(opts.commandConfigOpt)) else new Properties() - val channelSocketTimeoutMs = props.getProperty("channelSocketTimeoutMs", "600").toInt - val channelRetryBackoffMs = props.getProperty("channelRetryBackoffMsOpt", "300").toInt - if (!zkUtils.getConsumerGroups().contains(group)) - return (None, None) - - val topics = zkUtils.getTopicsByConsumerGroup(group) - val topicPartitions = getAllTopicPartitions(topics) - var groupConsumerIds = zkUtils.getConsumersInGroup(group) - - // mapping of topic partition -> consumer id - val consumerIdByTopicPartition = topicPartitions.map { topicPartition => - val owner = zkUtils.readDataMaybeNull(new ZKGroupTopicDirs(group, topicPartition.topic).consumerOwnerDir + "/" + topicPartition.partition)._1 - topicPartition -> owner.map(o => o.substring(0, o.lastIndexOf('-'))).getOrElse(MISSING_COLUMN_VALUE) - }.toMap - - // mapping of consumer id -> list of topic partitions - val consumerTopicPartitions = consumerIdByTopicPartition groupBy{_._2} map { - case (key, value) => (key, value.unzip._1.toArray) } - - // mapping of consumer id -> list of subscribed topics - val topicsByConsumerId = zkUtils.getTopicsPerMemberId(group) - - var assignmentRows = topicPartitions.flatMap { topicPartition => - val partitionOffsets = getPartitionOffsets(group, List(topicPartition), channelSocketTimeoutMs, channelRetryBackoffMs) - val consumerId = consumerIdByTopicPartition.get(topicPartition) - // since consumer id is repeated in client id, leave host and client id empty - consumerId.foreach(id => groupConsumerIds = groupConsumerIds.filterNot(_ == id)) - collectConsumerAssignment(group, None, List(topicPartition), partitionOffsets.get, consumerId, None, None) - } - - assignmentRows ++= groupConsumerIds.sortBy(- consumerTopicPartitions.get(_).size).flatMap { consumerId => - topicsByConsumerId(consumerId).flatMap { _ => - // since consumers with no topic partitions are processed here, we pass empty for topic partitions and offsets - // since consumer id is repeated in client id, leave host and client id empty - collectConsumerAssignment(group, None, Array[TopicPartition](), Map[TopicPartition, Option[Long]](), Some(consumerId), None, None) - } - } - - (None, Some(assignmentRows)) - } - - private def getAllTopicPartitions(topics: Seq[String]): Seq[TopicPartition] = { - val topicPartitionMap = zkUtils.getPartitionsForTopics(topics) - topics.flatMap { topic => - val partitions = topicPartitionMap.getOrElse(topic, Seq.empty) - partitions.map(new TopicPartition(topic, _)) - } - } - - protected def getLogEndOffsets(topicPartitions: Seq[TopicPartition]): Map[TopicPartition, LogOffsetResult] = { - topicPartitions.map { topicPartition => (topicPartition, - zkUtils.getLeaderForPartition(topicPartition.topic, topicPartition.partition) match { - case Some(-1) => LogOffsetResult.Unknown - case Some(brokerId) => - getZkConsumer(brokerId).map { consumer => - val topicAndPartition = new TopicAndPartition(topicPartition) - val request = OffsetRequest(Map(topicAndPartition -> PartitionOffsetRequestInfo(OffsetRequest.LatestTime, 1))) - val logEndOffset = consumer.getOffsetsBefore(request).partitionErrorAndOffsets(topicAndPartition).offsets.head - consumer.close() - LogOffsetResult.LogOffset(logEndOffset) - }.getOrElse(LogOffsetResult.Ignore) - case None => - printError(s"No broker for partition '$topicPartition'") - LogOffsetResult.Ignore - } - )}.toMap - } - - private def getPartitionOffsets(group: String, - topicPartitions: Seq[TopicPartition], - channelSocketTimeoutMs: Int, - channelRetryBackoffMs: Int): Map[TopicPartition, Long] = { - val offsetMap = mutable.Map[TopicAndPartition, Long]() - val channel = ClientUtils.channelToOffsetManager(group, zkUtils, channelSocketTimeoutMs, channelRetryBackoffMs) - channel.send(OffsetFetchRequest(group, topicPartitions.map(new TopicAndPartition(_)))) - val offsetFetchResponse = OffsetFetchResponse.readFrom(channel.receive().payload()) - - offsetFetchResponse.requestInfo.foreach { case (topicAndPartition, offsetAndMetadata) => - offsetAndMetadata match { - case 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(topicDirs.consumerOffsetDir + "/" + topicAndPartition.partition)._1.toLong - offsetMap.put(topicAndPartition, offset) - } catch { - case z: ZkNoNodeException => - printError(s"Could not fetch offset from zookeeper for group '$group' partition '$topicAndPartition' due to missing offset data in zookeeper.", Some(z)) - } - case offsetAndMetaData if offsetAndMetaData.error == Errors.NONE => - offsetMap.put(topicAndPartition, offsetAndMetadata.offset) - case _ => - printError(s"Could not fetch offset from kafka for group '$group' partition '$topicAndPartition' due to ${offsetAndMetadata.error.message}.") - } - } - channel.disconnect() - offsetMap.map { case (topicAndPartition, offset) => - (new TopicPartition(topicAndPartition.topic, topicAndPartition.partition), offset) - }.toMap - } - - private def deleteGroupsInfo(): Map[String, Errors] = { - val groups = opts.options.valuesOf(opts.groupOpt) - groups.asScala.map { group => - try { - if (AdminUtils.deleteConsumerGroupInZK(zkUtils, group)) { - println(s"Deleted all consumer group information for group '$group' in zookeeper.") - group -> Errors.NONE - } - else { - printError(s"Delete for group '$group' failed because its consumers are still active.") - group -> Errors.NON_EMPTY_GROUP - } - } - catch { - case e: ZkNoNodeException => - printError(s"Delete for group '$group' failed because group does not exist.", Some(e)) - group -> Errors.forException(e) - } - }.toMap - } - - private def deleteGroupsInfoForTopic(): Map[String, Errors] = { - val groups = opts.options.valuesOf(opts.groupOpt) - val topic = opts.options.valueOf(opts.topicOpt) - Topic.validate(topic) - groups.asScala.map { group => - try { - if (AdminUtils.deleteConsumerGroupInfoForTopicInZK(zkUtils, group, topic)) { - println(s"Deleted consumer group information for group '$group' topic '$topic' in zookeeper.") - group -> Errors.NONE - } - else { - printError(s"Delete for group '$group' topic '$topic' failed because its consumers are still active.") - group -> Errors.NON_EMPTY_GROUP - } - } - catch { - case e: ZkNoNodeException => - printError(s"Delete for group '$group' topic '$topic' failed because group does not exist.", Some(e)) - group -> Errors.forException(e) - } - }.toMap - } - - private def deleteAllGroupsInfoForTopic(): Map[String, Errors] = { - val topic = opts.options.valueOf(opts.topicOpt) - Topic.validate(topic) - val deletedGroups = AdminUtils.deleteAllConsumerGroupInfoForTopicInZK(zkUtils, topic) - println(s"Deleted consumer group information for all inactive consumer groups for topic '$topic' in zookeeper.") - deletedGroups.map(_ -> Errors.NONE).toMap - - } - - private def getZkConsumer(brokerId: Int): Option[SimpleConsumer] = { - try { - zkUtils.getBrokerInfo(brokerId) - .map(_.brokerEndPoint(ListenerName.forSecurityProtocol(SecurityProtocol.PLAINTEXT))) - .map(endPoint => new SimpleConsumer(endPoint.host, endPoint.port, 10000, 100000, "ConsumerGroupCommand")) - .orElse(throw new BrokerNotAvailableException("Broker id %d does not exist".format(brokerId))) - } catch { - case t: Throwable => - printError(s"Could not parse broker info due to ${t.getMessage}", Some(t)) - None - } - } - } - - class KafkaConsumerGroupService(val opts: ConsumerGroupCommandOptions) extends ConsumerGroupService { - - private val adminClient = createAdminClient() - - // `consumer` is only needed for `describe`, so we instantiate it lazily - private var consumer: KafkaConsumer[String, String] = _ - - override def listGroups(): List[String] = { - adminClient.listAllConsumerGroupsFlattened().map(_.groupId) - } - - override def collectGroupOffsets(): (Option[String], Option[Seq[PartitionAssignmentState]]) = { + private[admin] def collectGroupOffsets(): (Option[String], Option[Seq[PartitionAssignmentState]]) = { val group = opts.options.valueOf(opts.groupOpt) val consumerGroupSummary = adminClient.describeConsumerGroup(group, opts.options.valueOf(opts.timeoutMsOpt)) val assignments = consumerGroupSummary.consumers.map { consumers => @@ -587,7 +328,7 @@ object ConsumerGroupCommand extends Logging { (Some(consumerGroupSummary.state), assignments) } - override def collectGroupMembers(verbose: Boolean): (Option[String], Option[Seq[MemberAssignmentState]]) = { + private[admin] def collectGroupMembers(verbose: Boolean): (Option[String], Option[Seq[MemberAssignmentState]]) = { val group = opts.options.valueOf(opts.groupOpt) val consumerGroupSummary = adminClient.describeConsumerGroup(group, opts.options.valueOf(opts.timeoutMsOpt)) (Some(consumerGroupSummary.state), @@ -598,30 +339,34 @@ object ConsumerGroupCommand extends Logging { ) } - override def collectGroupState(): GroupState = { + private[admin] def collectGroupState(): GroupState = { val group = opts.options.valueOf(opts.groupOpt) val consumerGroupSummary = adminClient.describeConsumerGroup(group, opts.options.valueOf(opts.timeoutMsOpt)) GroupState(group, consumerGroupSummary.coordinator, consumerGroupSummary.assignmentStrategy, consumerGroupSummary.state, consumerGroupSummary.consumers.get.size) } - protected def getLogEndOffsets(topicPartitions: Seq[TopicPartition]): Map[TopicPartition, LogOffsetResult] = { + private def getLogEndOffsets(topicPartitions: Seq[TopicPartition]): Map[TopicPartition, LogOffsetResult] = { val offsets = getConsumer.endOffsets(topicPartitions.asJava) topicPartitions.map { topicPartition => - val logEndOffset = offsets.get(topicPartition) - topicPartition -> LogOffsetResult.LogOffset(logEndOffset) + Option(offsets.get(topicPartition)) match { + case Some(logEndOffset) => topicPartition -> LogOffsetResult.LogOffset(logEndOffset) + case _ => topicPartition -> LogOffsetResult.Unknown + } }.toMap } - protected def getLogStartOffsets(topicPartitions: Seq[TopicPartition]): Map[TopicPartition, LogOffsetResult] = { + private def getLogStartOffsets(topicPartitions: Seq[TopicPartition]): Map[TopicPartition, LogOffsetResult] = { val offsets = getConsumer.beginningOffsets(topicPartitions.asJava) topicPartitions.map { topicPartition => - val logStartOffset = offsets.get(topicPartition) - topicPartition -> LogOffsetResult.LogOffset(logStartOffset) + Option(offsets.get(topicPartition)) match { + case Some(logStartOffset) => topicPartition -> LogOffsetResult.LogOffset(logStartOffset) + case _ => topicPartition -> LogOffsetResult.Unknown + } }.toMap } - protected def getLogTimestampOffsets(topicPartitions: Seq[TopicPartition], timestamp: java.lang.Long): Map[TopicPartition, LogOffsetResult] = { + private def getLogTimestampOffsets(topicPartitions: Seq[TopicPartition], timestamp: java.lang.Long): Map[TopicPartition, LogOffsetResult] = { val consumer = getConsumer consumer.assign(topicPartitions.asJava) @@ -646,13 +391,13 @@ object ConsumerGroupCommand extends Logging { AdminClient.create(props) } - private def getConsumer = { + private def getConsumer: KafkaConsumer[String, String] = { if (consumer == null) - consumer = createNewConsumer() + consumer = createConsumer consumer } - private def createNewConsumer(): KafkaConsumer[String, String] = { + private def createConsumer: KafkaConsumer[String, String] = { val properties = new Properties() val deserializer = (new StringDeserializer).getClass.getName val brokerUrl = opts.options.valueOf(opts.bootstrapServerOpt) @@ -668,7 +413,7 @@ object ConsumerGroupCommand extends Logging { new KafkaConsumer(properties) } - override def resetOffsets(): Map[TopicPartition, OffsetAndMetadata] = { + def resetOffsets(): Map[TopicPartition, OffsetAndMetadata] = { val groupId = opts.options.valueOf(opts.groupOpt) val consumerGroupSummary = adminClient.describeConsumerGroup(groupId, opts.options.valueOf(opts.timeoutMsOpt)) consumerGroupSummary.state match { @@ -834,12 +579,12 @@ object ConsumerGroupCommand extends Logging { } } - override def exportOffsetsToReset(assignmentsToReset: Map[TopicPartition, OffsetAndMetadata]): String = { + def exportOffsetsToReset(assignmentsToReset: Map[TopicPartition, OffsetAndMetadata]): String = { val rows = assignmentsToReset.map { case (k,v) => s"${k.topic},${k.partition},${v.offset}" }(collection.breakOut): List[String] rows.foldRight("")(_ + "\n" + _) } - override def deleteGroups(): Map[String, Errors] = { + def deleteGroups(): Map[String, Errors] = { val groupsToDelete = opts.options.valuesOf(opts.groupOpt).asScala.toList val result = adminClient.deleteConsumerGroups(groupsToDelete) val successfullyDeleted = result.filter { @@ -871,9 +616,7 @@ object ConsumerGroupCommand extends Logging { } class ConsumerGroupCommandOptions(args: Array[String]) { - val ZkConnectDoc = "REQUIRED (for consumer groups based on the old consumer): The connection string for the zookeeper connection in the form host:port. " + - "Multiple URLS can be given to allow fail-over." - val BootstrapServerDoc = "REQUIRED (for consumer groups based on the new consumer): The server to connect to." + val BootstrapServerDoc = "REQUIRED: The server(s) to connect to." val GroupDoc = "The consumer group we wish to act on." val TopicDoc = "The topic whose consumer group information should be deleted or topic whose should be included in the reset offset process. " + "In `reset-offsets` case, partitions can be specified using this format: `topic1:0,1,2`, where 0,1,2 are the partition to be included in the process. " + @@ -883,12 +626,7 @@ object ConsumerGroupCommand extends Logging { val DescribeDoc = "Describe consumer group and list offset lag (number of messages not yet processed) 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: Group deletion only works for old ZK-based consumer groups, and one has to use it carefully to only delete groups that are not active." + "over the entire consumer group. For instance --group g1 --group g2" val TimeoutMsDoc = "The timeout that can be set for some use cases. For example, it can be used when describing the group " + "to specify the maximum amount of time in milliseconds to wait before the group stabilizes (when the group is just created, " + "or is going through some changes)." @@ -921,10 +659,6 @@ object ConsumerGroupCommand extends Logging { "Example: --bootstrap-server localhost:9092 --describe --group group1 --state" val parser = new OptionParser(false) - val zkConnectOpt = parser.accepts("zookeeper", ZkConnectDoc) - .withRequiredArg - .describedAs("urls") - .ofType(classOf[String]) val bootstrapServerOpt = parser.accepts("bootstrap-server", BootstrapServerDoc) .withRequiredArg .describedAs("server to connect to") @@ -979,21 +713,17 @@ object ConsumerGroupCommand extends Logging { .ofType(classOf[Long]) val membersOpt = parser.accepts("members", MembersDoc) .availableIf(describeOpt) - .availableUnless(zkConnectOpt) val verboseOpt = parser.accepts("verbose", VerboseDoc) .availableIf(describeOpt) - .availableUnless(zkConnectOpt) val offsetsOpt = parser.accepts("offsets", OffsetsDoc) .availableIf(describeOpt) - .availableUnless(zkConnectOpt) val stateOpt = parser.accepts("state", StateDoc) .availableIf(describeOpt) - .availableUnless(zkConnectOpt) + parser.mutuallyExclusive(membersOpt, offsetsOpt, stateOpt) val options = parser.parse(args : _*) - val useOldConsumer = options.has(zkConnectOpt) val describeOptPresent = options.has(describeOpt) val allConsumerGroupLevelOpts: Set[OptionSpec[_]] = Set(listOpt, describeOpt, deleteOpt, resetOffsetsOpt) @@ -1002,25 +732,20 @@ object ConsumerGroupCommand extends Logging { def checkArgs() { // check required args - if (options.has(timeoutMsOpt) && (!describeOptPresent || useOldConsumer)) - debug(s"Option $timeoutMsOpt is applicable only when both $bootstrapServerOpt and $describeOpt are used.") + if (options.has(timeoutMsOpt) && !describeOptPresent) + debug(s"Option $timeoutMsOpt is applicable only when $describeOpt is used.") - if (useOldConsumer) { - if (options.has(bootstrapServerOpt)) - CommandLineUtils.printUsageAndDie(parser, s"Option $bootstrapServerOpt is not valid with $zkConnectOpt.") - } else { - CommandLineUtils.checkRequiredArgs(parser, options, bootstrapServerOpt) + CommandLineUtils.checkRequiredArgs(parser, options, bootstrapServerOpt) - if (options.has(deleteOpt) && options.has(topicOpt)) - CommandLineUtils.printUsageAndDie(parser, s"When deleting a consumer group the option $topicOpt is only " + - s"valid with $zkConnectOpt. The new consumer does not support topic-specific offset deletion from a consumer group.") - } + if (options.has(deleteOpt) && options.has(topicOpt)) + CommandLineUtils.printUsageAndDie(parser, s"The consumer does not support topic-specific offset " + + "deletion from a consumer group.") if (describeOptPresent) CommandLineUtils.checkRequiredArgs(parser, options, groupOpt) - if (options.has(deleteOpt) && !options.has(groupOpt) && !options.has(topicOpt)) - CommandLineUtils.printUsageAndDie(parser, s"Option $deleteOpt either takes $groupOpt, $topicOpt, or both") + if (options.has(deleteOpt) && !options.has(groupOpt)) + CommandLineUtils.printUsageAndDie(parser, s"Option $deleteOpt takes $groupOpt") if (options.has(resetOffsetsOpt)) { if (options.has(dryRunOpt) && options.has(executeOpt)) diff --git a/core/src/main/scala/kafka/admin/TopicCommand.scala b/core/src/main/scala/kafka/admin/TopicCommand.scala index a9c61a97c3a4d..3914b564eda3f 100755 --- a/core/src/main/scala/kafka/admin/TopicCommand.scala +++ b/core/src/main/scala/kafka/admin/TopicCommand.scala @@ -22,7 +22,7 @@ import java.util.Properties import joptsimple._ import kafka.common.AdminCommandFailedException import kafka.utils.Implicits._ -import kafka.consumer.Whitelist +import kafka.utils.Whitelist import kafka.log.LogConfig import kafka.server.ConfigType import kafka.utils._ diff --git a/core/src/main/scala/kafka/api/ApiUtils.scala b/core/src/main/scala/kafka/api/ApiUtils.scala index 2145e8c5c672c..4a0c8b0c636b7 100644 --- a/core/src/main/scala/kafka/api/ApiUtils.scala +++ b/core/src/main/scala/kafka/api/ApiUtils.scala @@ -16,8 +16,9 @@ */ package kafka.api -import java.nio._ -import kafka.common._ +import java.nio.ByteBuffer + +import org.apache.kafka.common.KafkaException /** * Helper functions specific to parsing or serializing requests and responses @@ -75,26 +76,4 @@ object ApiUtils { } } - /** - * Read an integer out of the bytebuffer from the current position and check that it falls within the given - * range. If not, throw KafkaException. - */ - def readIntInRange(buffer: ByteBuffer, name: String, range: (Int, Int)): Int = { - val value = buffer.getInt - if(value < range._1 || value > range._2) - throw new KafkaException(name + " has value " + value + " which is not in the range " + range + ".") - else value - } - - /** - * Read a short out of the bytebuffer from the current position and check that it falls within the given - * range. If not, throw KafkaException. - */ - def readShortInRange(buffer: ByteBuffer, name: String, range: (Short, Short)): Short = { - val value = buffer.getShort - if(value < range._1 || value > range._2) - throw new KafkaException(name + " has value " + value + " which is not in the range " + range + ".") - else value - } - } diff --git a/core/src/main/scala/kafka/api/ApiVersion.scala b/core/src/main/scala/kafka/api/ApiVersion.scala index 9ed6432cbfd1d..5528b52e2b31b 100644 --- a/core/src/main/scala/kafka/api/ApiVersion.scala +++ b/core/src/main/scala/kafka/api/ApiVersion.scala @@ -70,9 +70,9 @@ object ApiVersion { // Introduced DeleteGroupsRequest V0 via KIP-229, plus KIP-227 incremental fetch requests, // and KafkaStorageException for fetch requests. KAFKA_1_1_IV0, - // Introduced OffsetsForLeaderEpochRequest V1 via KIP-279 + // Introduced OffsetsForLeaderEpochRequest V1 via KIP-279 (Fix log divergence between leader and follower after fast leader fail over) KAFKA_2_0_IV0, - // Introduced ApiVersionsRequest V2 via KIP-219 + // Several request versions were bumped due to KIP-219 (Improve quota communication) KAFKA_2_0_IV1 ) @@ -248,4 +248,4 @@ case object KAFKA_2_0_IV1 extends DefaultApiVersion { val subVersion = "IV1" val recordVersion = RecordVersion.V2 val id: Int = 16 -} \ 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 deleted file mode 100644 index 0379559e2db2c..0000000000000 --- a/core/src/main/scala/kafka/api/FetchRequest.scala +++ /dev/null @@ -1,270 +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 kafka.utils.nonthreadsafe -import kafka.api.ApiUtils._ -import kafka.common.TopicAndPartition -import kafka.consumer.ConsumerConfig -import java.util.concurrent.atomic.AtomicInteger -import java.nio.ByteBuffer - -import org.apache.kafka.common.protocol.ApiKeys - -import scala.collection.mutable.ArrayBuffer -import scala.util.Random - -case class PartitionFetchInfo(offset: Long, fetchSize: Int) - -@deprecated("This object has been deprecated and will be removed in a future release.", "0.11.0.0") -object FetchRequest { - - private val random = new Random - - val CurrentVersion = 3.shortValue - val DefaultMaxWait = 0 - val DefaultMinBytes = 0 - val DefaultMaxBytes = Int.MaxValue - val DefaultCorrelationId = 0 - - def readFrom(buffer: ByteBuffer): FetchRequest = { - val versionId = buffer.getShort - val correlationId = buffer.getInt - val clientId = readShortString(buffer) - val replicaId = buffer.getInt - val maxWait = buffer.getInt - val minBytes = buffer.getInt - val maxBytes = if (versionId < 3) DefaultMaxBytes else buffer.getInt - val topicCount = buffer.getInt - val pairs = (1 to topicCount).flatMap(_ => { - val topic = readShortString(buffer) - val partitionCount = buffer.getInt - (1 to partitionCount).map(_ => { - val partitionId = buffer.getInt - val offset = buffer.getLong - val fetchSize = buffer.getInt - (TopicAndPartition(topic, partitionId), PartitionFetchInfo(offset, fetchSize)) - }) - }) - FetchRequest(versionId, correlationId, clientId, replicaId, maxWait, minBytes, maxBytes, Vector(pairs:_*)) - } - - def shuffle(requestInfo: Seq[(TopicAndPartition, PartitionFetchInfo)]): Seq[(TopicAndPartition, PartitionFetchInfo)] = { - val groupedByTopic = requestInfo.groupBy { case (tp, _) => tp.topic }.map { case (topic, values) => - topic -> random.shuffle(values) - } - random.shuffle(groupedByTopic.toSeq).flatMap { case (_, partitions) => - partitions.map { case (tp, fetchInfo) => tp -> fetchInfo } - } - } - - def batchByTopic[T](s: Seq[(TopicAndPartition, T)]): Seq[(String, Seq[(Int, T)])] = { - val result = new ArrayBuffer[(String, ArrayBuffer[(Int, T)])] - s.foreach { case (TopicAndPartition(t, p), value) => - if (result.isEmpty || result.last._1 != t) - result += (t -> new ArrayBuffer) - result.last._2 += (p -> value) - } - result - } - -} - -@deprecated("This class has been deprecated and will be removed in a future release.", "0.11.0.0") -case class FetchRequest(versionId: Short = FetchRequest.CurrentVersion, - correlationId: Int = FetchRequest.DefaultCorrelationId, - clientId: String = ConsumerConfig.DefaultClientId, - replicaId: Int = Request.OrdinaryConsumerId, - maxWait: Int = FetchRequest.DefaultMaxWait, - minBytes: Int = FetchRequest.DefaultMinBytes, - maxBytes: Int = FetchRequest.DefaultMaxBytes, - requestInfo: Seq[(TopicAndPartition, PartitionFetchInfo)]) - extends RequestOrResponse(Some(ApiKeys.FETCH.id)) { - - /** - * Partitions the request info into a list of lists (one for each topic) while preserving request info ordering - */ - private type PartitionInfos = Seq[(Int, PartitionFetchInfo)] - private lazy val requestInfoGroupedByTopic: Seq[(String, PartitionInfos)] = FetchRequest.batchByTopic(requestInfo) - - /** Public constructor for the clients */ - @deprecated("The order of partitions in `requestInfo` is relevant, so this constructor is deprecated in favour of the " + - "one that takes a Seq", since = "0.10.1.0") - def this(correlationId: Int, - clientId: String, - maxWait: Int, - minBytes: Int, - maxBytes: Int, - requestInfo: Map[TopicAndPartition, PartitionFetchInfo]) { - this(versionId = FetchRequest.CurrentVersion, - correlationId = correlationId, - clientId = clientId, - replicaId = Request.OrdinaryConsumerId, - maxWait = maxWait, - minBytes = minBytes, - maxBytes = maxBytes, - requestInfo = FetchRequest.shuffle(requestInfo.toSeq)) - } - - /** Public constructor for the clients */ - def this(correlationId: Int, - clientId: String, - maxWait: Int, - minBytes: Int, - maxBytes: Int, - requestInfo: Seq[(TopicAndPartition, PartitionFetchInfo)]) { - this(versionId = FetchRequest.CurrentVersion, - correlationId = correlationId, - clientId = clientId, - replicaId = Request.OrdinaryConsumerId, - maxWait = maxWait, - minBytes = minBytes, - maxBytes = maxBytes, - requestInfo = requestInfo) - } - - def writeTo(buffer: ByteBuffer) { - buffer.putShort(versionId) - buffer.putInt(correlationId) - writeShortString(buffer, clientId) - buffer.putInt(replicaId) - buffer.putInt(maxWait) - buffer.putInt(minBytes) - if (versionId >= 3) - buffer.putInt(maxBytes) - buffer.putInt(requestInfoGroupedByTopic.size) // topic count - requestInfoGroupedByTopic.foreach { - case (topic, partitionFetchInfos) => - writeShortString(buffer, topic) - buffer.putInt(partitionFetchInfos.size) // partition count - partitionFetchInfos.foreach { - case (partition, PartitionFetchInfo(offset, fetchSize)) => - buffer.putInt(partition) - buffer.putLong(offset) - buffer.putInt(fetchSize) - } - } - } - - def sizeInBytes: Int = { - 2 + /* versionId */ - 4 + /* correlationId */ - shortStringLength(clientId) + - 4 + /* replicaId */ - 4 + /* maxWait */ - 4 + /* minBytes */ - (if (versionId >= 3) 4 /* maxBytes */ else 0) + - 4 + /* topic count */ - requestInfoGroupedByTopic.foldLeft(0)((foldedTopics, currTopic) => { - val (topic, partitionFetchInfos) = currTopic - foldedTopics + - shortStringLength(topic) + - 4 + /* partition count */ - partitionFetchInfos.size * ( - 4 + /* partition id */ - 8 + /* offset */ - 4 /* fetch size */ - ) - }) - } - - def isFromFollower = Request.isValidBrokerId(replicaId) - - def isFromOrdinaryConsumer = replicaId == Request.OrdinaryConsumerId - - def isFromLowLevelConsumer = replicaId == Request.DebuggingConsumerId - - def numPartitions = requestInfo.size - - override def toString: String = { - describe(true) - } - - override def describe(details: Boolean): String = { - val fetchRequest = new StringBuilder - fetchRequest.append("Name: " + this.getClass.getSimpleName) - fetchRequest.append("; Version: " + versionId) - fetchRequest.append("; CorrelationId: " + correlationId) - fetchRequest.append("; ClientId: " + clientId) - fetchRequest.append("; ReplicaId: " + replicaId) - fetchRequest.append("; MaxWait: " + maxWait + " ms") - fetchRequest.append("; MinBytes: " + minBytes + " bytes") - fetchRequest.append("; MaxBytes:" + maxBytes + " bytes") - if(details) - fetchRequest.append("; RequestInfo: " + requestInfo.mkString(",")) - fetchRequest.toString() - } -} - -@deprecated("This class has been deprecated and will be removed in a future release.", "0.11.0.0") -@nonthreadsafe -class FetchRequestBuilder() { - private val correlationId = new AtomicInteger(0) - private var versionId = FetchRequest.CurrentVersion - private var clientId = ConsumerConfig.DefaultClientId - private var replicaId = Request.OrdinaryConsumerId - private var maxWait = FetchRequest.DefaultMaxWait - private var minBytes = FetchRequest.DefaultMinBytes - private var maxBytes = FetchRequest.DefaultMaxBytes - private val requestMap = new collection.mutable.ArrayBuffer[(TopicAndPartition, PartitionFetchInfo)] - - def addFetch(topic: String, partition: Int, offset: Long, fetchSize: Int) = { - requestMap.append((TopicAndPartition(topic, partition), PartitionFetchInfo(offset, fetchSize))) - this - } - - def clientId(clientId: String): FetchRequestBuilder = { - this.clientId = clientId - this - } - - /** - * Only for internal use. Clients shouldn't set replicaId. - */ - private[kafka] def replicaId(replicaId: Int): FetchRequestBuilder = { - this.replicaId = replicaId - this - } - - def maxWait(maxWait: Int): FetchRequestBuilder = { - this.maxWait = maxWait - this - } - - def minBytes(minBytes: Int): FetchRequestBuilder = { - this.minBytes = minBytes - this - } - - def maxBytes(maxBytes: Int): FetchRequestBuilder = { - this.maxBytes = maxBytes - this - } - - def requestVersion(versionId: Short): FetchRequestBuilder = { - this.versionId = versionId - this - } - - def build() = { - val fetchRequest = FetchRequest(versionId, correlationId.getAndIncrement, clientId, replicaId, maxWait, minBytes, - maxBytes, new ArrayBuffer() ++ requestMap) - requestMap.clear() - fetchRequest - } -} diff --git a/core/src/main/scala/kafka/api/FetchResponse.scala b/core/src/main/scala/kafka/api/FetchResponse.scala deleted file mode 100644 index ae2f19c4bcc1f..0000000000000 --- a/core/src/main/scala/kafka/api/FetchResponse.scala +++ /dev/null @@ -1,174 +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.common.TopicAndPartition -import kafka.message.{ByteBufferMessageSet, MessageSet} -import kafka.api.ApiUtils._ -import org.apache.kafka.common.protocol.Errors - -import scala.collection._ - -object FetchResponsePartitionData { - def readFrom(buffer: ByteBuffer): FetchResponsePartitionData = { - val error = Errors.forCode(buffer.getShort) - val hw = buffer.getLong - val messageSetSize = buffer.getInt - val messageSetBuffer = buffer.slice() - messageSetBuffer.limit(messageSetSize) - buffer.position(buffer.position() + messageSetSize) - new FetchResponsePartitionData(error, hw, new ByteBufferMessageSet(messageSetBuffer)) - } - - val headerSize = - 2 + /* error code */ - 8 + /* high watermark */ - 4 /* messageSetSize */ -} - -case class FetchResponsePartitionData(error: Errors = Errors.NONE, hw: Long = -1L, messages: MessageSet) { - val sizeInBytes = FetchResponsePartitionData.headerSize + messages.sizeInBytes -} - -object TopicData { - def readFrom(buffer: ByteBuffer): TopicData = { - val topic = readShortString(buffer) - val partitionCount = buffer.getInt - val topicPartitionDataPairs = (1 to partitionCount).map(_ => { - val partitionId = buffer.getInt - val partitionData = FetchResponsePartitionData.readFrom(buffer) - (partitionId, partitionData) - }) - TopicData(topic, Seq(topicPartitionDataPairs:_*)) - } - - def headerSize(topic: String) = - shortStringLength(topic) + - 4 /* partition count */ -} - -case class TopicData(topic: String, partitionData: Seq[(Int, FetchResponsePartitionData)]) { - val sizeInBytes = - TopicData.headerSize(topic) + partitionData.foldLeft(0)((folded, data) => { - folded + data._2.sizeInBytes + 4 - } /*_ + _.sizeInBytes + 4*/) - - val headerSize = TopicData.headerSize(topic) -} - -@deprecated("This object has been deprecated and will be removed in a future release.", "1.0.0") -object 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) - topicData.partitionData.map { case (partitionId, partitionData) => - (TopicAndPartition(topicData.topic, partitionId), partitionData) - } - }) - FetchResponse(correlationId, Vector(pairs:_*), requestVersion, throttleTime) - } - - type FetchResponseEntry = (Int, FetchResponsePartitionData) - - def batchByTopic(data: Seq[(TopicAndPartition, FetchResponsePartitionData)]): Seq[(String, Seq[FetchResponseEntry])] = - FetchRequest.batchByTopic(data) - - // 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: Seq[(String, Seq[FetchResponseEntry])], - requestVersion: Int): Int = { - headerSize(requestVersion) + - dataGroupedByTopic.foldLeft(0) { case (folded, (topic, partitionDataMap)) => - val topicData = TopicData(topic, partitionDataMap.map { - case (partitionId, partitionData) => (partitionId, partitionData) - }) - folded + topicData.sizeInBytes - } - } -} - -@deprecated("This object has been deprecated and will be removed in a future release.", "1.0.0") -case class FetchResponse(correlationId: Int, - data: Seq[(TopicAndPartition, FetchResponsePartitionData)], - requestVersion: Int = 0, - throttleTimeMs: Int = 0) - extends RequestOrResponse() { - - /** - * Partitions the data into a map of maps (one for each topic). - */ - private lazy val dataByTopicAndPartition = data.toMap - lazy val dataGroupedByTopic = FetchResponse.batchByTopic(data) - val headerSizeInBytes = FetchResponse.headerSize(requestVersion) - lazy val sizeInBytes = FetchResponse.responseSize(dataGroupedByTopic, requestVersion) - - /* - * 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) - dataByTopicAndPartition.get(topicAndPartition) match { - case Some(partitionData) => partitionData - case _ => - throw new IllegalArgumentException( - "No partition %s in fetch response %s".format(topicAndPartition, this.toString)) - } - } - - def messageSet(topic: String, partition: Int): ByteBufferMessageSet = - partitionDataFor(topic, partition).messages.asInstanceOf[ByteBufferMessageSet] - - def highWatermark(topic: String, partition: Int) = partitionDataFor(topic, partition).hw - - def hasError = dataByTopicAndPartition.values.exists(_.error != Errors.NONE) - - def error(topic: String, partition: Int) = partitionDataFor(topic, partition).error -} diff --git a/core/src/main/scala/kafka/api/GroupCoordinatorRequest.scala b/core/src/main/scala/kafka/api/GroupCoordinatorRequest.scala deleted file mode 100644 index 6ee6ae73ffecf..0000000000000 --- a/core/src/main/scala/kafka/api/GroupCoordinatorRequest.scala +++ /dev/null @@ -1,74 +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 org.apache.kafka.common.protocol.ApiKeys - -@deprecated("This object has been deprecated and will be removed in a future release.", "1.0.0") -object GroupCoordinatorRequest { - val CurrentVersion = 0.shortValue - val DefaultClientId = "" - - def readFrom(buffer: ByteBuffer) = { - // envelope - val versionId = buffer.getShort - val correlationId = buffer.getInt - val clientId = ApiUtils.readShortString(buffer) - - // request - val group = ApiUtils.readShortString(buffer) - GroupCoordinatorRequest(group, versionId, correlationId, clientId) - } - -} - -@deprecated("This object has been deprecated and will be removed in a future release.", "1.0.0") -case class GroupCoordinatorRequest(group: String, - versionId: Short = GroupCoordinatorRequest.CurrentVersion, - correlationId: Int = 0, - clientId: String = GroupCoordinatorRequest.DefaultClientId) - extends RequestOrResponse(Some(ApiKeys.FIND_COORDINATOR.id)) { - - def sizeInBytes = - 2 + /* versionId */ - 4 + /* correlationId */ - ApiUtils.shortStringLength(clientId) + - ApiUtils.shortStringLength(group) - - def writeTo(buffer: ByteBuffer) { - // envelope - buffer.putShort(versionId) - buffer.putInt(correlationId) - ApiUtils.writeShortString(buffer, clientId) - - // consumer metadata request - ApiUtils.writeShortString(buffer, group) - } - - def describe(details: Boolean) = { - val consumerMetadataRequest = new StringBuilder - consumerMetadataRequest.append("Name: " + this.getClass.getSimpleName) - consumerMetadataRequest.append("; Version: " + versionId) - consumerMetadataRequest.append("; CorrelationId: " + correlationId) - consumerMetadataRequest.append("; ClientId: " + clientId) - consumerMetadataRequest.append("; Group: " + group) - consumerMetadataRequest.toString() - } -} diff --git a/core/src/main/scala/kafka/api/GroupCoordinatorResponse.scala b/core/src/main/scala/kafka/api/GroupCoordinatorResponse.scala deleted file mode 100644 index 7fd5fe3de6775..0000000000000 --- a/core/src/main/scala/kafka/api/GroupCoordinatorResponse.scala +++ /dev/null @@ -1,60 +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.cluster.BrokerEndPoint -import org.apache.kafka.common.protocol.Errors - -@deprecated("This object has been deprecated and will be removed in a future release.", "1.0.0") -object GroupCoordinatorResponse { - val CurrentVersion = 0 - - private val NoBrokerEndpointOpt = Some(BrokerEndPoint(id = -1, host = "", port = -1)) - - def readFrom(buffer: ByteBuffer) = { - val correlationId = buffer.getInt - val error = Errors.forCode(buffer.getShort) - val broker = BrokerEndPoint.readFrom(buffer) - val coordinatorOpt = if (error == Errors.NONE) - Some(broker) - else - None - - GroupCoordinatorResponse(coordinatorOpt, error, correlationId) - } - -} - -@deprecated("This object has been deprecated and will be removed in a future release.", "1.0.0") -case class GroupCoordinatorResponse (coordinatorOpt: Option[BrokerEndPoint], error: Errors, correlationId: Int) - extends RequestOrResponse() { - - def sizeInBytes = - 4 + /* correlationId */ - 2 + /* error code */ - coordinatorOpt.orElse(GroupCoordinatorResponse.NoBrokerEndpointOpt).get.sizeInBytes - - def writeTo(buffer: ByteBuffer) { - buffer.putInt(correlationId) - buffer.putShort(error.code) - coordinatorOpt.orElse(GroupCoordinatorResponse.NoBrokerEndpointOpt).foreach(_.writeTo(buffer)) - } - - def describe(details: Boolean) = toString -} diff --git a/core/src/main/scala/kafka/api/OffsetCommitRequest.scala b/core/src/main/scala/kafka/api/OffsetCommitRequest.scala deleted file mode 100644 index bffcec3fad49f..0000000000000 --- a/core/src/main/scala/kafka/api/OffsetCommitRequest.scala +++ /dev/null @@ -1,181 +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.api.ApiUtils._ -import kafka.common.{OffsetAndMetadata, TopicAndPartition} -import kafka.utils.Logging -import org.apache.kafka.common.protocol.ApiKeys - -import scala.collection._ - -@deprecated("This object has been deprecated and will be removed in a future release.", "1.0.0") -object OffsetCommitRequest extends Logging { - val CurrentVersion: Short = 2 - val DefaultClientId = "" - - def readFrom(buffer: ByteBuffer): OffsetCommitRequest = { - // Read values from the envelope - val versionId = buffer.getShort - assert(versionId == 0 || versionId == 1 || versionId == 2, - "Version " + versionId + " is invalid for OffsetCommitRequest. Valid versions are 0, 1 or 2.") - - val correlationId = buffer.getInt - val clientId = readShortString(buffer) - - // Read the OffsetRequest - val groupId = 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 memberId: String = - if (versionId >= 1) - readShortString(buffer) - else - org.apache.kafka.common.requests.OffsetCommitRequest.DEFAULT_MEMBER_ID - - // version 2 specific fields - val retentionMs: Long = - if (versionId >= 2) - buffer.getLong - else - org.apache.kafka.common.requests.OffsetCommitRequest.DEFAULT_RETENTION_TIME - - val topicCount = buffer.getInt - val pairs = (1 to topicCount).flatMap(_ => { - val topic = readShortString(buffer) - val partitionCount = buffer.getInt - (1 to partitionCount).map(_ => { - val partitionId = buffer.getInt - val offset = buffer.getLong - val timestamp = { - // version 1 specific field - if (versionId == 1) - buffer.getLong - else - org.apache.kafka.common.requests.OffsetCommitRequest.DEFAULT_TIMESTAMP - } - val metadata = readShortString(buffer) - - (TopicAndPartition(topic, partitionId), OffsetAndMetadata(offset, metadata, timestamp)) - }) - }) - - OffsetCommitRequest(groupId, immutable.Map(pairs:_*), versionId, correlationId, clientId, groupGenerationId, memberId, retentionMs) - } -} - -@deprecated("This object has been deprecated and will be removed in a future release.", "1.0.0") -case class OffsetCommitRequest(groupId: String, - requestInfo: immutable.Map[TopicAndPartition, OffsetAndMetadata], - versionId: Short = OffsetCommitRequest.CurrentVersion, - correlationId: Int = 0, - clientId: String = OffsetCommitRequest.DefaultClientId, - groupGenerationId: Int = org.apache.kafka.common.requests.OffsetCommitRequest.DEFAULT_GENERATION_ID, - memberId: String = org.apache.kafka.common.requests.OffsetCommitRequest.DEFAULT_MEMBER_ID, - retentionMs: Long = org.apache.kafka.common.requests.OffsetCommitRequest.DEFAULT_RETENTION_TIME) - extends RequestOrResponse(Some(ApiKeys.OFFSET_COMMIT.id)) { - - assert(versionId == 0 || versionId == 1 || versionId == 2, - "Version " + versionId + " is invalid for OffsetCommitRequest. Valid versions are 0, 1 or 2.") - - lazy val requestInfoGroupedByTopic = requestInfo.groupBy(_._1.topic) - - def writeTo(buffer: ByteBuffer) { - // Write envelope - buffer.putShort(versionId) - buffer.putInt(correlationId) - writeShortString(buffer, clientId) - - // Write OffsetCommitRequest - writeShortString(buffer, groupId) // consumer group - - // version 1 and 2 specific data - if (versionId >= 1) { - buffer.putInt(groupGenerationId) - writeShortString(buffer, memberId) - } - - // version 2 or above specific data - if (versionId >= 2) { - buffer.putLong(retentionMs) - } - - buffer.putInt(requestInfoGroupedByTopic.size) // number of topics - requestInfoGroupedByTopic.foreach( t1 => { // topic -> Map[TopicAndPartition, OffsetMetadataAndError] - writeShortString(buffer, t1._1) // topic - buffer.putInt(t1._2.size) // number of partitions for this topic - t1._2.foreach( t2 => { - buffer.putInt(t2._1.partition) - buffer.putLong(t2._2.offset) - // version 1 specific data - if (versionId == 1) - buffer.putLong(t2._2.commitTimestamp) - writeShortString(buffer, t2._2.metadata) - }) - }) - } - - override def sizeInBytes = - 2 + /* versionId */ - 4 + /* correlationId */ - shortStringLength(clientId) + - shortStringLength(groupId) + - (if (versionId >= 1) 4 /* group generation id */ + shortStringLength(memberId) else 0) + - (if (versionId >= 2) 8 /* retention time */ else 0) + - 4 + /* topic count */ - requestInfoGroupedByTopic.foldLeft(0)((count, topicAndOffsets) => { - val (topic, offsets) = topicAndOffsets - count + - shortStringLength(topic) + /* topic */ - 4 + /* number of partitions */ - offsets.foldLeft(0)((innerCount, offsetAndMetadata) => { - innerCount + - 4 /* partition */ + - 8 /* offset */ + - (if (versionId == 1) 8 else 0) /* timestamp */ + - shortStringLength(offsetAndMetadata._2.metadata) - }) - }) - - override def describe(details: Boolean): String = { - val offsetCommitRequest = new StringBuilder - offsetCommitRequest.append("Name: " + this.getClass.getSimpleName) - offsetCommitRequest.append("; Version: " + versionId) - offsetCommitRequest.append("; CorrelationId: " + correlationId) - offsetCommitRequest.append("; ClientId: " + clientId) - offsetCommitRequest.append("; GroupId: " + groupId) - offsetCommitRequest.append("; GroupGenerationId: " + groupGenerationId) - offsetCommitRequest.append("; MemberId: " + memberId) - offsetCommitRequest.append("; RetentionMs: " + retentionMs) - if(details) - offsetCommitRequest.append("; RequestInfo: " + requestInfo.mkString(",")) - offsetCommitRequest.toString() - } - - override def toString = { - describe(details = true) - } -} diff --git a/core/src/main/scala/kafka/api/OffsetCommitResponse.scala b/core/src/main/scala/kafka/api/OffsetCommitResponse.scala deleted file mode 100644 index 07adc76234f8e..0000000000000 --- a/core/src/main/scala/kafka/api/OffsetCommitResponse.scala +++ /dev/null @@ -1,82 +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.utils.Logging -import kafka.common.TopicAndPartition -import org.apache.kafka.common.protocol.Errors - -@deprecated("This object has been deprecated and will be removed in a future release.", "1.0.0") -object OffsetCommitResponse extends Logging { - val CurrentVersion: Short = 0 - - def readFrom(buffer: ByteBuffer): OffsetCommitResponse = { - val correlationId = buffer.getInt - val topicCount = buffer.getInt - val pairs = (1 to topicCount).flatMap(_ => { - val topic = ApiUtils.readShortString(buffer) - val partitionCount = buffer.getInt - (1 to partitionCount).map(_ => { - val partitionId = buffer.getInt - val error = Errors.forCode(buffer.getShort) - (TopicAndPartition(topic, partitionId), error) - }) - }) - OffsetCommitResponse(Map(pairs:_*), correlationId) - } -} - -@deprecated("This object has been deprecated and will be removed in a future release.", "1.0.0") -case class OffsetCommitResponse(commitStatus: Map[TopicAndPartition, Errors], - correlationId: Int = 0) - extends RequestOrResponse() { - - lazy val commitStatusGroupedByTopic = commitStatus.groupBy(_._1.topic) - - def hasError = commitStatus.values.exists(_ != Errors.NONE) - - def writeTo(buffer: ByteBuffer) { - buffer.putInt(correlationId) - buffer.putInt(commitStatusGroupedByTopic.size) - commitStatusGroupedByTopic.foreach { case(topic, statusMap) => - ApiUtils.writeShortString(buffer, topic) - buffer.putInt(statusMap.size) // partition count - statusMap.foreach { case(topicAndPartition, error) => - buffer.putInt(topicAndPartition.partition) - buffer.putShort(error.code) - } - } - } - - override def sizeInBytes = - 4 + /* correlationId */ - 4 + /* topic count */ - commitStatusGroupedByTopic.foldLeft(0)((count, partitionStatusMap) => { - val (topic, partitionStatus) = partitionStatusMap - count + - ApiUtils.shortStringLength(topic) + - 4 + /* partition count */ - partitionStatus.size * ( 4 /* partition */ + 2 /* error code */) - }) - - override def describe(details: Boolean):String = { toString } - -} - diff --git a/core/src/main/scala/kafka/api/OffsetFetchRequest.scala b/core/src/main/scala/kafka/api/OffsetFetchRequest.scala deleted file mode 100644 index c24078d52fe06..0000000000000 --- a/core/src/main/scala/kafka/api/OffsetFetchRequest.scala +++ /dev/null @@ -1,108 +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.api.ApiUtils._ -import kafka.common.TopicAndPartition -import kafka.utils.Logging -import org.apache.kafka.common.protocol.ApiKeys - -@deprecated("This object has been deprecated and will be removed in a future release.", "1.0.0") -object OffsetFetchRequest extends Logging { - val CurrentVersion: Short = 2 - val DefaultClientId = "" - - def readFrom(buffer: ByteBuffer): OffsetFetchRequest = { - // Read values from the envelope - val versionId = buffer.getShort - val correlationId = buffer.getInt - val clientId = readShortString(buffer) - - // Read the OffsetFetchRequest - val consumerGroupId = readShortString(buffer) - val topicCount = buffer.getInt - val pairs = (1 to topicCount).flatMap(_ => { - val topic = readShortString(buffer) - val partitionCount = buffer.getInt - (1 to partitionCount).map(_ => { - val partitionId = buffer.getInt - TopicAndPartition(topic, partitionId) - }) - }) - OffsetFetchRequest(consumerGroupId, pairs, versionId, correlationId, clientId) - } -} - -@deprecated("This object has been deprecated and will be removed in a future release.", "1.0.0") -case class OffsetFetchRequest(groupId: String, - requestInfo: Seq[TopicAndPartition], - versionId: Short = OffsetFetchRequest.CurrentVersion, - correlationId: Int = 0, - clientId: String = OffsetFetchRequest.DefaultClientId) - extends RequestOrResponse(Some(ApiKeys.OFFSET_FETCH.id)) { - - lazy val requestInfoGroupedByTopic = requestInfo.groupBy(_.topic) - - def writeTo(buffer: ByteBuffer) { - // Write envelope - buffer.putShort(versionId) - buffer.putInt(correlationId) - writeShortString(buffer, clientId) - - // Write OffsetFetchRequest - writeShortString(buffer, groupId) // consumer group - buffer.putInt(requestInfoGroupedByTopic.size) // number of topics - requestInfoGroupedByTopic.foreach( t1 => { // (topic, Seq[TopicAndPartition]) - writeShortString(buffer, t1._1) // topic - buffer.putInt(t1._2.size) // number of partitions for this topic - t1._2.foreach( t2 => { - buffer.putInt(t2.partition) - }) - }) - } - - override def sizeInBytes = - 2 + /* versionId */ - 4 + /* correlationId */ - shortStringLength(clientId) + - shortStringLength(groupId) + - 4 + /* topic count */ - requestInfoGroupedByTopic.foldLeft(0)((count, t) => { - count + shortStringLength(t._1) + /* topic */ - 4 + /* number of partitions */ - t._2.size * 4 /* partition */ - }) - - override def describe(details: Boolean): String = { - val offsetFetchRequest = new StringBuilder - offsetFetchRequest.append("Name: " + this.getClass.getSimpleName) - offsetFetchRequest.append("; Version: " + versionId) - offsetFetchRequest.append("; CorrelationId: " + correlationId) - offsetFetchRequest.append("; ClientId: " + clientId) - offsetFetchRequest.append("; GroupId: " + groupId) - if (details) - offsetFetchRequest.append("; RequestInfo: " + requestInfo.mkString(",")) - offsetFetchRequest.toString() - } - - override def toString: String = { - describe(details = true) - } -} diff --git a/core/src/main/scala/kafka/api/OffsetFetchResponse.scala b/core/src/main/scala/kafka/api/OffsetFetchResponse.scala deleted file mode 100644 index b875dcc5a01e2..0000000000000 --- a/core/src/main/scala/kafka/api/OffsetFetchResponse.scala +++ /dev/null @@ -1,107 +0,0 @@ -/** - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package kafka.api - -import java.nio.ByteBuffer - -import kafka.api.ApiUtils._ -import kafka.common.{TopicAndPartition, OffsetMetadataAndError} -import kafka.utils.Logging - -import org.apache.kafka.common.protocol.Errors - -@deprecated("This object has been deprecated and will be removed in a future release.", "1.0.0") -object OffsetFetchResponse extends Logging { - - def readFrom(buffer: ByteBuffer): OffsetFetchResponse = { - readFrom(buffer, OffsetFetchRequest.CurrentVersion) - } - - def readFrom(buffer: ByteBuffer, requestVersion: Int): OffsetFetchResponse = { - val correlationId = buffer.getInt - val topicCount = buffer.getInt - val pairs = (1 to topicCount).flatMap(_ => { - val topic = readShortString(buffer) - val partitionCount = buffer.getInt - (1 to partitionCount).map(_ => { - val partitionId = buffer.getInt - val offset = buffer.getLong - val metadata = readShortString(buffer) - val error = Errors.forCode(buffer.getShort) - (TopicAndPartition(topic, partitionId), OffsetMetadataAndError(offset, metadata, error)) - }) - }) - - val error = requestVersion match { - case 0 | 1 => Errors.NONE - case _ => Errors.forCode(buffer.getShort) - } - - OffsetFetchResponse(Map(pairs:_*), requestVersion, correlationId, error) - } -} - -@deprecated("This object has been deprecated and will be removed in a future release.", "1.0.0") -case class OffsetFetchResponse(requestInfo: Map[TopicAndPartition, OffsetMetadataAndError], - requestVersion: Int = OffsetFetchRequest.CurrentVersion, - correlationId: Int = 0, - error: Errors = Errors.NONE) - extends RequestOrResponse() { - - lazy val requestInfoGroupedByTopic = requestInfo.groupBy(_._1.topic) - - def writeTo(buffer: ByteBuffer) { - buffer.putInt(correlationId) - buffer.putInt(requestInfoGroupedByTopic.size) // number of topics - requestInfoGroupedByTopic.foreach( t1 => { // topic -> Map[TopicAndPartition, OffsetMetadataAndError] - writeShortString(buffer, t1._1) // topic - buffer.putInt(t1._2.size) // number of partitions for this topic - t1._2.foreach( t2 => { // TopicAndPartition -> OffsetMetadataAndError - buffer.putInt(t2._1.partition) - buffer.putLong(t2._2.offset) - writeShortString(buffer, t2._2.metadata) - buffer.putShort(t2._2.error.code) - }) - }) - - // the top level error_code was introduced in v2 - if (requestVersion > 1) - buffer.putShort(error.code) - } - - override def sizeInBytes = - 4 + /* correlationId */ - 4 + /* topic count */ - requestInfoGroupedByTopic.foldLeft(0)((count, topicAndOffsets) => { - val (topic, offsets) = topicAndOffsets - count + - shortStringLength(topic) + /* topic */ - 4 + /* number of partitions */ - offsets.foldLeft(0)((innerCount, offsetsAndMetadata) => { - innerCount + - 4 /* partition */ + - 8 /* offset */ + - shortStringLength(offsetsAndMetadata._2.metadata) + - 2 /* error */ - }) - }) + - (if (requestVersion > 1) 2 else 0) /* error */ - - override def describe(details: Boolean):String = { toString } -} - diff --git a/core/src/main/scala/kafka/api/OffsetRequest.scala b/core/src/main/scala/kafka/api/OffsetRequest.scala deleted file mode 100644 index f5483b1acf7e2..0000000000000 --- a/core/src/main/scala/kafka/api/OffsetRequest.scala +++ /dev/null @@ -1,126 +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.api.ApiUtils._ -import kafka.common.TopicAndPartition -import org.apache.kafka.common.protocol.ApiKeys - -@deprecated("This object has been deprecated and will be removed in a future release.", "1.0.0") -object OffsetRequest { - val CurrentVersion = 0.shortValue - val DefaultClientId = "" - - val SmallestTimeString = "smallest" - val LargestTimeString = "largest" - val LatestTime = -1L - val EarliestTime = -2L - - def readFrom(buffer: ByteBuffer): OffsetRequest = { - val versionId = buffer.getShort - val correlationId = buffer.getInt - val clientId = readShortString(buffer) - val replicaId = buffer.getInt - val topicCount = buffer.getInt - val pairs = (1 to topicCount).flatMap(_ => { - val topic = readShortString(buffer) - val partitionCount = buffer.getInt - (1 to partitionCount).map(_ => { - val partitionId = buffer.getInt - val time = buffer.getLong - val maxNumOffsets = buffer.getInt - (TopicAndPartition(topic, partitionId), PartitionOffsetRequestInfo(time, maxNumOffsets)) - }) - }) - OffsetRequest(Map(pairs:_*), versionId= versionId, clientId = clientId, correlationId = correlationId, replicaId = replicaId) - } -} - -case class PartitionOffsetRequestInfo(time: Long, maxNumOffsets: Int) - -@deprecated("This object has been deprecated and will be removed in a future release.", "1.0.0") -case class OffsetRequest(requestInfo: Map[TopicAndPartition, PartitionOffsetRequestInfo], - versionId: Short = OffsetRequest.CurrentVersion, - correlationId: Int = 0, - clientId: String = OffsetRequest.DefaultClientId, - replicaId: Int = Request.OrdinaryConsumerId) - extends RequestOrResponse(Some(ApiKeys.LIST_OFFSETS.id)) { - - def this(requestInfo: Map[TopicAndPartition, PartitionOffsetRequestInfo], correlationId: Int, replicaId: Int) = this(requestInfo, OffsetRequest.CurrentVersion, correlationId, OffsetRequest.DefaultClientId, replicaId) - - lazy val requestInfoGroupedByTopic = requestInfo.groupBy(_._1.topic) - - def writeTo(buffer: ByteBuffer) { - buffer.putShort(versionId) - buffer.putInt(correlationId) - writeShortString(buffer, clientId) - buffer.putInt(replicaId) - - buffer.putInt(requestInfoGroupedByTopic.size) // topic count - requestInfoGroupedByTopic.foreach { - case((topic, partitionInfos)) => - writeShortString(buffer, topic) - buffer.putInt(partitionInfos.size) // partition count - partitionInfos.foreach { - case (TopicAndPartition(_, partition), partitionInfo) => - buffer.putInt(partition) - buffer.putLong(partitionInfo.time) - buffer.putInt(partitionInfo.maxNumOffsets) - } - } - } - - def sizeInBytes = - 2 + /* versionId */ - 4 + /* correlationId */ - shortStringLength(clientId) + - 4 + /* replicaId */ - 4 + /* topic count */ - requestInfoGroupedByTopic.foldLeft(0)((foldedTopics, currTopic) => { - val (topic, partitionInfos) = currTopic - foldedTopics + - shortStringLength(topic) + - 4 + /* partition count */ - partitionInfos.size * ( - 4 + /* partition */ - 8 + /* time */ - 4 /* maxNumOffsets */ - ) - }) - - def isFromOrdinaryClient = replicaId == Request.OrdinaryConsumerId - def isFromDebuggingClient = replicaId == Request.DebuggingConsumerId - - override def toString: String = { - describe(true) - } - - override def describe(details: Boolean): String = { - val offsetRequest = new StringBuilder - offsetRequest.append("Name: " + this.getClass.getSimpleName) - offsetRequest.append("; Version: " + versionId) - offsetRequest.append("; CorrelationId: " + correlationId) - offsetRequest.append("; ClientId: " + clientId) - offsetRequest.append("; ReplicaId: " + replicaId) - if(details) - offsetRequest.append("; RequestInfo: " + requestInfo.mkString(",")) - offsetRequest.toString() - } -} diff --git a/core/src/main/scala/kafka/api/OffsetResponse.scala b/core/src/main/scala/kafka/api/OffsetResponse.scala deleted file mode 100644 index 95a31d501b459..0000000000000 --- a/core/src/main/scala/kafka/api/OffsetResponse.scala +++ /dev/null @@ -1,102 +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.common.TopicAndPartition -import kafka.api.ApiUtils._ -import org.apache.kafka.common.protocol.Errors - - -@deprecated("This object has been deprecated and will be removed in a future release.", "1.0.0") -object OffsetResponse { - - def readFrom(buffer: ByteBuffer): OffsetResponse = { - val correlationId = buffer.getInt - val numTopics = buffer.getInt - val pairs = (1 to numTopics).flatMap(_ => { - val topic = readShortString(buffer) - val numPartitions = buffer.getInt - (1 to numPartitions).map(_ => { - val partition = buffer.getInt - val error = Errors.forCode(buffer.getShort) - val numOffsets = buffer.getInt - val offsets = (1 to numOffsets).map(_ => buffer.getLong) - (TopicAndPartition(topic, partition), PartitionOffsetsResponse(error, offsets)) - }) - }) - OffsetResponse(correlationId, Map(pairs:_*)) - } - -} - - -case class PartitionOffsetsResponse(error: Errors, offsets: Seq[Long]) { - override def toString: String = { - new String("error: " + error.exceptionName + " offsets: " + offsets.mkString) - } -} - - -@deprecated("This object has been deprecated and will be removed in a future release.", "1.0.0") -case class OffsetResponse(correlationId: Int, - partitionErrorAndOffsets: Map[TopicAndPartition, PartitionOffsetsResponse]) - extends RequestOrResponse() { - - lazy val offsetsGroupedByTopic = partitionErrorAndOffsets.groupBy(_._1.topic) - - def hasError = partitionErrorAndOffsets.values.exists(_.error != Errors.NONE) - - val sizeInBytes = { - 4 + /* correlation id */ - 4 + /* topic count */ - offsetsGroupedByTopic.foldLeft(0)((foldedTopics, currTopic) => { - val (topic, errorAndOffsetsMap) = currTopic - foldedTopics + - shortStringLength(topic) + - 4 + /* partition count */ - errorAndOffsetsMap.foldLeft(0)((foldedPartitions, currPartition) => { - foldedPartitions + - 4 + /* partition id */ - 2 + /* partition error */ - 4 + /* offset array length */ - currPartition._2.offsets.size * 8 /* offset */ - }) - }) - } - - def writeTo(buffer: ByteBuffer) { - buffer.putInt(correlationId) - buffer.putInt(offsetsGroupedByTopic.size) // topic count - offsetsGroupedByTopic.foreach { - case((topic, errorAndOffsetsMap)) => - writeShortString(buffer, topic) - buffer.putInt(errorAndOffsetsMap.size) // partition count - errorAndOffsetsMap.foreach { - case((TopicAndPartition(_, partition), errorAndOffsets)) => - buffer.putInt(partition) - buffer.putShort(errorAndOffsets.error.code) - buffer.putInt(errorAndOffsets.offsets.size) // offset array length - errorAndOffsets.offsets.foreach(buffer.putLong(_)) - } - } - } - - override def describe(details: Boolean):String = { toString } -} - diff --git a/core/src/main/scala/kafka/common/InvalidMessageSizeException.scala b/core/src/main/scala/kafka/api/Request.scala similarity index 76% rename from core/src/main/scala/kafka/common/InvalidMessageSizeException.scala rename to core/src/main/scala/kafka/api/Request.scala index 6a7bb47cc3bf4..b6ec2735e9df6 100644 --- a/core/src/main/scala/kafka/common/InvalidMessageSizeException.scala +++ b/core/src/main/scala/kafka/api/Request.scala @@ -15,12 +15,13 @@ * limitations under the License. */ -package kafka.common +package kafka.api -/** - * Indicates the client has requested a range no longer available on the server - */ -class InvalidMessageSizeException(message: String) extends RuntimeException(message) { - def this() = this(null) -} +object Request { + val OrdinaryConsumerId: Int = -1 + val DebuggingConsumerId: Int = -2 + val FutureLocalReplicaId: Int = -3 + // Broker ids are non-negative int. + def isValidBrokerId(brokerId: Int): Boolean = brokerId >= 0 +} diff --git a/core/src/main/scala/kafka/api/RequestOrResponse.scala b/core/src/main/scala/kafka/api/RequestOrResponse.scala deleted file mode 100644 index a0fe129c40abe..0000000000000 --- a/core/src/main/scala/kafka/api/RequestOrResponse.scala +++ /dev/null @@ -1,48 +0,0 @@ -package kafka.api - -/** - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT 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 java.nio._ -import kafka.network.RequestChannel -import kafka.utils.Logging - -object Request { - val OrdinaryConsumerId: Int = -1 - val DebuggingConsumerId: Int = -2 - val FutureLocalReplicaId: Int = -3 - - // Broker ids are non-negative int. - def isValidBrokerId(brokerId: Int): Boolean = brokerId >= 0 -} - - -abstract class RequestOrResponse(val requestId: Option[Short] = None) extends Logging { - - def sizeInBytes: Int - - def writeTo(buffer: ByteBuffer): Unit - - def handleError(e: Throwable, requestChannel: RequestChannel, request: RequestChannel.Request): Unit = {} - - /* The purpose of this API is to return a string description of the Request mainly for the purpose of request logging. - * This API has no meaning for a Response object. - * @param details If this is false, omit the parts of the request description that are proportional to the number of - * topics or partitions. This is mainly to control the amount of request logging. */ - def describe(details: Boolean): String -} - diff --git a/core/src/main/scala/kafka/api/TopicMetadata.scala b/core/src/main/scala/kafka/api/TopicMetadata.scala deleted file mode 100644 index e4d730c95af6d..0000000000000 --- a/core/src/main/scala/kafka/api/TopicMetadata.scala +++ /dev/null @@ -1,153 +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 kafka.cluster.BrokerEndPoint -import java.nio.ByteBuffer -import kafka.api.ApiUtils._ -import kafka.utils.Logging -import org.apache.kafka.common.protocol.Errors - -object TopicMetadata { - - val NoLeaderNodeId = -1 - - def readFrom(buffer: ByteBuffer, brokers: Map[Int, BrokerEndPoint]): TopicMetadata = { - val error = Errors.forCode(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(i) = partitionMetadata - } - new TopicMetadata(topic, partitionsMetadata, error) - } -} - -case class TopicMetadata(topic: String, partitionsMetadata: Seq[PartitionMetadata], error: Errors = Errors.NONE) extends Logging { - def sizeInBytes: Int = { - 2 /* error code */ + - shortStringLength(topic) + - 4 + partitionsMetadata.map(_.sizeInBytes).sum /* size and partition data array */ - } - - def writeTo(buffer: ByteBuffer) { - /* error code */ - buffer.putShort(error.code) - /* topic */ - writeShortString(buffer, topic) - /* number of partitions */ - buffer.putInt(partitionsMetadata.size) - partitionsMetadata.foreach(m => m.writeTo(buffer)) - } - - override def toString: String = { - val topicMetadataInfo = new StringBuilder - topicMetadataInfo.append("{TopicMetadata for topic %s -> ".format(topic)) - error match { - case Errors.NONE => - partitionsMetadata.foreach { partitionMetadata => - partitionMetadata.error match { - case Errors.NONE => - topicMetadataInfo.append("\nMetadata for partition [%s,%d] is %s".format(topic, - partitionMetadata.partitionId, partitionMetadata.toString())) - case Errors.REPLICA_NOT_AVAILABLE => - // this error message means some replica other than the leader is not available. The consumer - // doesn't care about non leader replicas, so ignore this - topicMetadataInfo.append("\nMetadata for partition [%s,%d] is %s".format(topic, - partitionMetadata.partitionId, partitionMetadata.toString())) - case error: Errors => - topicMetadataInfo.append("\nMetadata for partition [%s,%d] is not available due to %s".format(topic, - partitionMetadata.partitionId, error.exceptionName)) - } - } - case error: Errors => - topicMetadataInfo.append("\nNo partition metadata for topic %s due to %s".format(topic, - error.exceptionName)) - } - topicMetadataInfo.append("}") - topicMetadataInfo.toString() - } -} - -object PartitionMetadata { - - def readFrom(buffer: ByteBuffer, brokers: Map[Int, BrokerEndPoint]): PartitionMetadata = { - val error = Errors.forCode(readShortInRange(buffer, "error code", (-1, Short.MaxValue))) - val partitionId = readIntInRange(buffer, "partition id", (0, Int.MaxValue)) /* partition id */ - val leaderId = buffer.getInt - val leader = brokers.get(leaderId) - - /* list of all replicas */ - val numReplicas = readIntInRange(buffer, "number of all replicas", (0, Int.MaxValue)) - val replicaIds = (0 until numReplicas).map(_ => buffer.getInt) - val replicas = replicaIds.map(brokers) - - /* list of in-sync replicas */ - val numIsr = readIntInRange(buffer, "number of in-sync replicas", (0, Int.MaxValue)) - val isrIds = (0 until numIsr).map(_ => buffer.getInt) - val isr = isrIds.map(brokers) - - new PartitionMetadata(partitionId, leader, replicas, isr, error) - } -} - -case class PartitionMetadata(partitionId: Int, - leader: Option[BrokerEndPoint], - replicas: Seq[BrokerEndPoint], - isr: Seq[BrokerEndPoint] = Seq.empty, - error: Errors = Errors.NONE) extends Logging { - def sizeInBytes: Int = { - 2 /* error code */ + - 4 /* partition id */ + - 4 /* leader */ + - 4 + 4 * replicas.size /* replica array */ + - 4 + 4 * isr.size /* isr array */ - } - - def writeTo(buffer: ByteBuffer) { - buffer.putShort(error.code) - buffer.putInt(partitionId) - - /* leader */ - val leaderId = leader.fold(TopicMetadata.NoLeaderNodeId)(leader => leader.id) - buffer.putInt(leaderId) - - /* number of replicas */ - buffer.putInt(replicas.size) - replicas.foreach(r => buffer.putInt(r.id)) - - /* number of in-sync replicas */ - buffer.putInt(isr.size) - isr.foreach(r => buffer.putInt(r.id)) - } - - override def toString: String = { - val partitionMetadataString = new StringBuilder - partitionMetadataString.append("\tpartition " + partitionId) - partitionMetadataString.append("\tleader: " + leader.getOrElse("none")) - partitionMetadataString.append("\treplicas: " + replicas.mkString(",")) - partitionMetadataString.append("\tisr: " + isr.mkString(",")) - partitionMetadataString.append("\tisUnderReplicated: " + (isr.size < replicas.size)) - partitionMetadataString.toString() - } - -} - - diff --git a/core/src/main/scala/kafka/api/TopicMetadataRequest.scala b/core/src/main/scala/kafka/api/TopicMetadataRequest.scala deleted file mode 100644 index 217cedc495983..0000000000000 --- a/core/src/main/scala/kafka/api/TopicMetadataRequest.scala +++ /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 kafka.api - -import java.nio.ByteBuffer - -import kafka.api.ApiUtils._ -import kafka.utils.Logging -import org.apache.kafka.common.protocol.ApiKeys - -@deprecated("This object has been deprecated and will be removed in a future release.", "1.0.0") -object TopicMetadataRequest extends Logging { - val CurrentVersion = 0.shortValue - val DefaultClientId = "" -} - -@deprecated("This object has been deprecated and will be removed in a future release.", "1.0.0") -case class TopicMetadataRequest(versionId: Short, - correlationId: Int, - clientId: String, - topics: Seq[String]) - extends RequestOrResponse(Some(ApiKeys.METADATA.id)){ - - def this(topics: Seq[String], correlationId: Int) = - this(TopicMetadataRequest.CurrentVersion, correlationId, TopicMetadataRequest.DefaultClientId, topics) - - def writeTo(buffer: ByteBuffer) { - buffer.putShort(versionId) - buffer.putInt(correlationId) - writeShortString(buffer, clientId) - buffer.putInt(topics.size) - topics.foreach(topic => writeShortString(buffer, topic)) - } - - def sizeInBytes: Int = { - 2 + /* version id */ - 4 + /* correlation id */ - shortStringLength(clientId) + /* client id */ - 4 + /* number of topics */ - topics.foldLeft(0)(_ + shortStringLength(_)) /* topics */ - } - - override def toString: String = { - describe(true) - } - - override def describe(details: Boolean): String = { - val topicMetadataRequest = new StringBuilder - topicMetadataRequest.append("Name: " + this.getClass.getSimpleName) - topicMetadataRequest.append("; Version: " + versionId) - topicMetadataRequest.append("; CorrelationId: " + correlationId) - topicMetadataRequest.append("; ClientId: " + clientId) - if(details) - topicMetadataRequest.append("; Topics: " + topics.mkString(",")) - topicMetadataRequest.toString() - } -} diff --git a/core/src/main/scala/kafka/api/TopicMetadataResponse.scala b/core/src/main/scala/kafka/api/TopicMetadataResponse.scala deleted file mode 100644 index 5447ea00daa22..0000000000000 --- a/core/src/main/scala/kafka/api/TopicMetadataResponse.scala +++ /dev/null @@ -1,57 +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 kafka.cluster.BrokerEndPoint -import java.nio.ByteBuffer - -@deprecated("This object has been deprecated and will be removed in a future release.", "1.0.0") -object TopicMetadataResponse { - - def readFrom(buffer: ByteBuffer): TopicMetadataResponse = { - val correlationId = buffer.getInt - val brokerCount = buffer.getInt - val brokers = (0 until brokerCount).map(_ => BrokerEndPoint.readFrom(buffer)) - val brokerMap = brokers.map(b => (b.id, b)).toMap - val topicCount = buffer.getInt - val topicsMetadata = (0 until topicCount).map(_ => TopicMetadata.readFrom(buffer, brokerMap)) - new TopicMetadataResponse(brokers, topicsMetadata, correlationId) - } -} - -@deprecated("This object has been deprecated and will be removed in a future release.", "1.0.0") -case class TopicMetadataResponse(brokers: Seq[BrokerEndPoint], - topicsMetadata: Seq[TopicMetadata], - correlationId: Int) - extends RequestOrResponse() { - val sizeInBytes: Int = { - 4 + 4 + brokers.map(_.sizeInBytes).sum + 4 + topicsMetadata.map(_.sizeInBytes).sum - } - - def writeTo(buffer: ByteBuffer) { - buffer.putInt(correlationId) - /* brokers */ - buffer.putInt(brokers.size) - brokers.foreach(_.writeTo(buffer)) - /* topic metadata */ - buffer.putInt(topicsMetadata.length) - topicsMetadata.foreach(_.writeTo(buffer)) - } - - override def describe(details: Boolean):String = { toString } -} diff --git a/core/src/main/scala/kafka/client/ClientUtils.scala b/core/src/main/scala/kafka/client/ClientUtils.scala deleted file mode 100755 index 53f38956d4b30..0000000000000 --- a/core/src/main/scala/kafka/client/ClientUtils.scala +++ /dev/null @@ -1,203 +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.client - -import org.apache.kafka.common.protocol.Errors - -import scala.collection._ -import kafka.cluster._ -import kafka.api._ -import kafka.common.{BrokerEndPointNotAvailableException, KafkaException} -import kafka.utils.{CoreUtils, Logging} - -import util.Random -import kafka.network.BlockingChannel -import kafka.utils.ZkUtils -import java.io.IOException - -import kafka.consumer.SimpleConsumer -import org.apache.kafka.common.security.auth.SecurityProtocol - - /** - * Helper functions common to clients (producer, consumer, or admin) - */ -@deprecated("This class has been deprecated and will be removed in a future release.", "0.11.0.0") -object ClientUtils extends Logging { - - /** - * Send a metadata request - * @param topics The topics for which the metadata needs to be fetched - * @param brokers The brokers in the cluster as configured on the client - * @param clientId The client's identifier - * @return topic metadata response - */ - def fetchTopicMetadata(topics: Set[String], brokers: Seq[BrokerEndPoint], clientId: String, timeoutMs: Int, - correlationId: Int = 0): TopicMetadataResponse = { - var fetchMetaDataSucceeded: Boolean = false - var i: Int = 0 - val topicMetadataRequest = new TopicMetadataRequest(TopicMetadataRequest.CurrentVersion, correlationId, clientId, - topics.toSeq) - var topicMetadataResponse: TopicMetadataResponse = null - var t: Throwable = null - // shuffle the list of brokers before sending metadata requests so that most requests don't get routed to the - // same broker - val shuffledBrokers = Random.shuffle(brokers) - while(i < shuffledBrokers.size && !fetchMetaDataSucceeded) { - val broker = shuffledBrokers(i) - val consumer = new SimpleConsumer(broker.host, broker.port, timeoutMs, BlockingChannel.UseDefaultBufferSize, - clientId) - info("Fetching metadata from broker %s with correlation id %d for %d topic(s) %s".format(shuffledBrokers(i), - correlationId, topics.size, topics)) - try { - topicMetadataResponse = consumer.send(topicMetadataRequest) - fetchMetaDataSucceeded = true - } - catch { - case e: Throwable => - warn("Fetching topic metadata with correlation id %d for topics [%s] from broker [%s] failed" - .format(correlationId, topics, shuffledBrokers(i).toString), e) - t = e - } finally { - i = i + 1 - consumer.close() - } - } - if (!fetchMetaDataSucceeded) { - throw new KafkaException("fetching topic metadata for topics [%s] from broker [%s] failed".format(topics, shuffledBrokers), t) - } else { - debug("Successfully fetched metadata for %d topic(s) %s".format(topics.size, topics)) - } - topicMetadataResponse - } - - /** - * Parse a list of broker urls in the form host1:port1, host2:port2, ... - */ - def parseBrokerList(brokerListStr: String): Seq[BrokerEndPoint] = { - val brokersStr = CoreUtils.parseCsvList(brokerListStr) - - brokersStr.zipWithIndex.map { case (address, brokerId) => - BrokerEndPoint.createBrokerEndPoint(brokerId, address) - } - } - - /** - * Creates a blocking channel to a random broker - */ - def channelToAnyBroker(zkUtils: ZkUtils, socketTimeoutMs: Int = 3000) : BlockingChannel = { - var channel: BlockingChannel = null - var connected = false - while (!connected) { - val allBrokers = getPlaintextBrokerEndPoints(zkUtils) - Random.shuffle(allBrokers).find { broker => - trace("Connecting to broker %s:%d.".format(broker.host, broker.port)) - try { - channel = new BlockingChannel(broker.host, broker.port, BlockingChannel.UseDefaultBufferSize, BlockingChannel.UseDefaultBufferSize, socketTimeoutMs) - channel.connect() - debug("Created channel to broker %s:%d.".format(channel.host, channel.port)) - true - } catch { - case _: Exception => - if (channel != null) channel.disconnect() - channel = null - info("Error while creating channel to %s:%d.".format(broker.host, broker.port)) - false - } - } - connected = channel != null - } - - channel - } - - /** - * Returns the first end point from each broker with the PLAINTEXT security protocol. - */ - def getPlaintextBrokerEndPoints(zkUtils: ZkUtils): Seq[BrokerEndPoint] = { - zkUtils.getAllBrokersInCluster().map { broker => - broker.endPoints.collectFirst { - case endPoint if endPoint.securityProtocol == SecurityProtocol.PLAINTEXT => - new BrokerEndPoint(broker.id, endPoint.host, endPoint.port) - }.getOrElse(throw new BrokerEndPointNotAvailableException(s"End point with security protocol PLAINTEXT not found for broker ${broker.id}")) - } - } - - /** - * Creates a blocking channel to the offset manager of the given group - */ - def channelToOffsetManager(group: String, zkUtils: ZkUtils, socketTimeoutMs: Int = 3000, retryBackOffMs: Int = 1000) = { - var queryChannel = channelToAnyBroker(zkUtils) - - var offsetManagerChannelOpt: Option[BlockingChannel] = None - - while (offsetManagerChannelOpt.isEmpty) { - - var coordinatorOpt: Option[BrokerEndPoint] = None - - while (coordinatorOpt.isEmpty) { - try { - if (!queryChannel.isConnected) - queryChannel = channelToAnyBroker(zkUtils) - debug("Querying %s:%d to locate offset manager for %s.".format(queryChannel.host, queryChannel.port, group)) - queryChannel.send(GroupCoordinatorRequest(group)) - val response = queryChannel.receive() - val consumerMetadataResponse = GroupCoordinatorResponse.readFrom(response.payload()) - debug("Consumer metadata response: " + consumerMetadataResponse.toString) - if (consumerMetadataResponse.error == Errors.NONE) - coordinatorOpt = consumerMetadataResponse.coordinatorOpt - else { - debug("Query to %s:%d to locate offset manager for %s failed - will retry in %d milliseconds." - .format(queryChannel.host, queryChannel.port, group, retryBackOffMs)) - Thread.sleep(retryBackOffMs) - } - } - catch { - case _: IOException => - info("Failed to fetch consumer metadata from %s:%d.".format(queryChannel.host, queryChannel.port)) - queryChannel.disconnect() - } - } - - val coordinator = coordinatorOpt.get - if (coordinator.host == queryChannel.host && coordinator.port == queryChannel.port) { - offsetManagerChannelOpt = Some(queryChannel) - } else { - val connectString = "%s:%d".format(coordinator.host, coordinator.port) - var offsetManagerChannel: BlockingChannel = null - try { - debug("Connecting to offset manager %s.".format(connectString)) - offsetManagerChannel = new BlockingChannel(coordinator.host, coordinator.port, - BlockingChannel.UseDefaultBufferSize, - BlockingChannel.UseDefaultBufferSize, - socketTimeoutMs) - offsetManagerChannel.connect() - offsetManagerChannelOpt = Some(offsetManagerChannel) - queryChannel.disconnect() - } - catch { - case _: IOException => // offsets manager may have moved - info("Error while connecting to %s.".format(connectString)) - if (offsetManagerChannel != null) offsetManagerChannel.disconnect() - Thread.sleep(retryBackOffMs) - offsetManagerChannelOpt = None // just in case someone decides to change shutdownChannel to not swallow exceptions - } - } - } - - offsetManagerChannelOpt.get - } - } diff --git a/core/src/main/scala/kafka/cluster/BrokerEndPoint.scala b/core/src/main/scala/kafka/cluster/BrokerEndPoint.scala index 847e959d76f71..986d35252a6b9 100644 --- a/core/src/main/scala/kafka/cluster/BrokerEndPoint.scala +++ b/core/src/main/scala/kafka/cluster/BrokerEndPoint.scala @@ -19,7 +19,7 @@ package kafka.cluster import java.nio.ByteBuffer import kafka.api.ApiUtils._ -import kafka.common.KafkaException +import org.apache.kafka.common.KafkaException import org.apache.kafka.common.utils.Utils._ object BrokerEndPoint { diff --git a/core/src/main/scala/kafka/cluster/EndPoint.scala b/core/src/main/scala/kafka/cluster/EndPoint.scala index 57ef0da7047fa..2bca5c8797673 100644 --- a/core/src/main/scala/kafka/cluster/EndPoint.scala +++ b/core/src/main/scala/kafka/cluster/EndPoint.scala @@ -17,7 +17,7 @@ package kafka.cluster -import kafka.common.KafkaException +import org.apache.kafka.common.KafkaException import org.apache.kafka.common.network.ListenerName import org.apache.kafka.common.security.auth.SecurityProtocol import org.apache.kafka.common.utils.Utils diff --git a/core/src/main/scala/kafka/cluster/Partition.scala b/core/src/main/scala/kafka/cluster/Partition.scala index b9180a45378a1..60f05ceb741c3 100755 --- a/core/src/main/scala/kafka/cluster/Partition.scala +++ b/core/src/main/scala/kafka/cluster/Partition.scala @@ -20,8 +20,8 @@ package kafka.cluster import java.util.concurrent.locks.ReentrantReadWriteLock import com.yammer.metrics.core.Gauge -import kafka.api.LeaderAndIsr -import kafka.api.Request +import kafka.api.{LeaderAndIsr, Request} +import kafka.common.UnexpectedAppendOffsetException import kafka.controller.KafkaController import kafka.log.{LogAppendInfo, LogConfig} import kafka.metrics.KafkaMetricsGroup @@ -30,7 +30,7 @@ import kafka.utils.CoreUtils.{inReadLock, inWriteLock} import kafka.utils._ import kafka.zk.AdminZkClient import org.apache.kafka.common.TopicPartition -import org.apache.kafka.common.errors.{NotEnoughReplicasException, NotLeaderForPartitionException, PolicyViolationException} +import org.apache.kafka.common.errors.{ReplicaNotAvailableException, NotEnoughReplicasException, NotLeaderForPartitionException, PolicyViolationException} import org.apache.kafka.common.protocol.Errors import org.apache.kafka.common.protocol.Errors._ import org.apache.kafka.common.record.MemoryRecords @@ -62,6 +62,9 @@ class Partition(val topic: String, private val leaderIsrUpdateLock = new ReentrantReadWriteLock private var zkVersion: Int = LeaderAndIsr.initialZKVersion @volatile private var leaderEpoch: Int = LeaderAndIsr.initialLeaderEpoch - 1 + // start offset for 'leaderEpoch' above (leader epoch of the current leader for this partition), + // defined when this broker is leader for partition + @volatile private var leaderEpochStartOffsetOpt: Option[Long] = None @volatile var leaderReplicaIdOpt: Option[Int] = None @volatile var inSyncReplicas: Set[Replica] = Set.empty[Replica] @@ -149,10 +152,10 @@ class Partition(val topic: String, * @return true iff the future replica is created */ def maybeCreateFutureReplica(logDir: String): Boolean = { - // The readLock is needed to make sure that while the caller checks the log directory of the + // The writeLock is needed to make sure that while the caller checks the log directory of the // current replica and the existence of the future replica, no other thread can update the log directory of the // current replica or remove the future replica. - inReadLock(leaderIsrUpdateLock) { + inWriteLock(leaderIsrUpdateLock) { val currentReplica = getReplica().get if (currentReplica.log.get.dir.getParent == logDir) false @@ -187,6 +190,10 @@ class Partition(val topic: String, def getReplica(replicaId: Int = localBrokerId): Option[Replica] = Option(allReplicasMap.get(replicaId)) + def getReplicaOrException(replicaId: Int = localBrokerId): Replica = + getReplica(replicaId).getOrElse( + throw new ReplicaNotAvailableException(s"Replica $replicaId is not available for partition $topicPartition")) + def leaderReplicaIfLocal: Option[Replica] = leaderReplicaIdOpt.filter(_ == localBrokerId).flatMap(getReplica) @@ -203,29 +210,52 @@ class Partition(val topic: String, allReplicasMap.remove(replicaId) } - def removeFutureLocalReplica() { + def futureReplicaDirChanged(newDestinationDir: String): Boolean = { + inReadLock(leaderIsrUpdateLock) { + getReplica(Request.FutureLocalReplicaId) match { + case Some(futureReplica) => + if (futureReplica.log.get.dir.getParent != newDestinationDir) + true + else + false + case None => false + } + } + } + + def removeFutureLocalReplica(deleteFromLogDir: Boolean = true) { inWriteLock(leaderIsrUpdateLock) { allReplicasMap.remove(Request.FutureLocalReplicaId) + if (deleteFromLogDir) + logManager.asyncDelete(topicPartition, isFuture = true) } } - // Return true iff the future log has caught up with the current log for this partition + // Return true iff the future replica exists and it has caught up with the current replica for this partition // Only ReplicaAlterDirThread will call this method and ReplicaAlterDirThread should remove the partition // from its partitionStates if this method returns true def maybeReplaceCurrentWithFutureReplica(): Boolean = { val replica = getReplica().get - val futureReplica = getReplica(Request.FutureLocalReplicaId).get - if (replica.logEndOffset == futureReplica.logEndOffset) { + val futureReplicaLEO = getReplica(Request.FutureLocalReplicaId).map(_.logEndOffset.messageOffset) + if (futureReplicaLEO.contains(replica.logEndOffset.messageOffset)) { // The write lock is needed to make sure that while ReplicaAlterDirThread checks the LEO of the // current replica, no other thread can update LEO of the current replica via log truncation or log append operation. inWriteLock(leaderIsrUpdateLock) { - if (replica.logEndOffset == futureReplica.logEndOffset) { - logManager.replaceCurrentWithFutureLog(topicPartition) - replica.log = futureReplica.log - futureReplica.log = None - allReplicasMap.remove(Request.FutureLocalReplicaId) - true - } else false + getReplica(Request.FutureLocalReplicaId) match { + case Some(futureReplica) => + if (replica.logEndOffset.messageOffset == futureReplica.logEndOffset.messageOffset) { + logManager.replaceCurrentWithFutureLog(topicPartition) + replica.log = futureReplica.log + futureReplica.log = None + allReplicasMap.remove(Request.FutureLocalReplicaId) + true + } else false + case None => + // Future replica is removed by a non-ReplicaAlterLogDirsThread before this method is called + // In this case the partition should have been removed from state of the ReplicaAlterLogDirsThread + // Return false so that ReplicaAlterLogDirsThread does not have to remove this partition from the state again to avoid race condition + false + } } } else false } @@ -236,9 +266,11 @@ class Partition(val topic: String, allReplicasMap.clear() inSyncReplicas = Set.empty[Replica] leaderReplicaIdOpt = None + leaderEpochStartOffsetOpt = None removePartitionMetrics() logManager.asyncDelete(topicPartition) - logManager.asyncDelete(topicPartition, isFuture = true) + if (logManager.getLog(topicPartition, isFuture = true).isDefined) + logManager.asyncDelete(topicPartition, isFuture = true) } } @@ -260,18 +292,28 @@ class Partition(val topic: String, // remove assigned replicas that have been removed by the controller (assignedReplicas.map(_.brokerId) -- newAssignedReplicas).foreach(removeReplica) inSyncReplicas = newInSyncReplicas + newAssignedReplicas.foreach(id => getOrCreateReplica(id, partitionStateInfo.isNew)) + val leaderReplica = getReplica().get + val leaderEpochStartOffset = leaderReplica.logEndOffset.messageOffset info(s"$topicPartition starts at Leader Epoch ${partitionStateInfo.basePartitionState.leaderEpoch} from " + - s"offset ${getReplica().get.logEndOffset.messageOffset}. Previous Leader Epoch was: $leaderEpoch") + s"offset $leaderEpochStartOffset. Previous Leader Epoch was: $leaderEpoch") //We cache the leader epoch here, persisting it only if it's local (hence having a log dir) leaderEpoch = partitionStateInfo.basePartitionState.leaderEpoch - newAssignedReplicas.foreach(id => getOrCreateReplica(id, partitionStateInfo.isNew)) - + leaderEpochStartOffsetOpt = Some(leaderEpochStartOffset) zkVersion = partitionStateInfo.basePartitionState.zkVersion - val isNewLeader = leaderReplicaIdOpt.map(_ != localBrokerId).getOrElse(true) - val leaderReplica = getReplica().get + // In the case of successive leader elections in a short time period, a follower may have + // entries in its log from a later epoch than any entry in the new leader's log. In order + // to ensure that these followers can truncate to the right offset, we must cache the new + // leader epoch and the start offset since it should be larger than any epoch that a follower + // would try to query. + leaderReplica.epochs.foreach { epochCache => + epochCache.assign(leaderEpoch, leaderEpochStartOffset) + } + + val isNewLeader = !leaderReplicaIdOpt.contains(localBrokerId) val curLeaderLogEndOffset = leaderReplica.logEndOffset.messageOffset val curTimeMs = time.milliseconds // initialize lastCaughtUpTime of replicas as well as their lastFetchTimeMs and lastFetchLeaderLogEndOffset. @@ -299,12 +341,15 @@ class Partition(val topic: String, /** * Make the local replica the follower by setting the new leader and ISR to empty - * If the leader replica id does not change, return false to indicate the replica manager + * If the leader replica id does not change and the new epoch is equal or one + * greater (that is, no updates have been missed), return false to indicate to the + * replica manager that state is already correct and the become-follower steps can be skipped */ def makeFollower(controllerId: Int, partitionStateInfo: LeaderAndIsrRequest.PartitionState, correlationId: Int): Boolean = { inWriteLock(leaderIsrUpdateLock) { val newAssignedReplicas = partitionStateInfo.basePartitionState.replicas.asScala.map(_.toInt) - val newLeaderBrokerId: Int = partitionStateInfo.basePartitionState.leader + val newLeaderBrokerId = partitionStateInfo.basePartitionState.leader + val oldLeaderEpoch = leaderEpoch // record the epoch of the controller that made the leadership decision. This is useful while updating the isr // to maintain the decision maker controller's epoch in the zookeeper path controllerEpoch = partitionStateInfo.basePartitionState.controllerEpoch @@ -314,9 +359,12 @@ class Partition(val topic: String, (assignedReplicas.map(_.brokerId) -- newAssignedReplicas).foreach(removeReplica) inSyncReplicas = Set.empty[Replica] leaderEpoch = partitionStateInfo.basePartitionState.leaderEpoch + leaderEpochStartOffsetOpt = None zkVersion = partitionStateInfo.basePartitionState.zkVersion - if (leaderReplicaIdOpt.isDefined && leaderReplicaIdOpt.get == newLeaderBrokerId) { + // If the leader is unchanged and the epochs are no more than one change apart, indicate that no follower changes are required + // Otherwise, we missed a leader epoch update, which means the leader's log may have been truncated prior to the current epoch. + if (leaderReplicaIdOpt.contains(newLeaderBrokerId) && (leaderEpoch == oldLeaderEpoch || leaderEpoch == oldLeaderEpoch + 1)) { false } else { @@ -356,7 +404,11 @@ class Partition(val topic: String, /** * Check and maybe expand the ISR of the partition. - * A replica will be added to ISR if its LEO >= current hw of the partition. + * A replica will be added to ISR if its LEO >= current hw of the partition and it is caught up to + * an offset within the current leader epoch. A replica must be caught up to the current leader + * epoch before it can join ISR, because otherwise, if there is committed data between current + * leader's HW and LEO, the replica may become the leader before it fetches the committed data + * and the data will be lost. * * Technically, a replica shouldn't be in ISR if it hasn't caught up for longer than replicaLagTimeMaxMs, * even if its log end offset is >= HW. However, to be consistent with how the follower determines @@ -373,9 +425,11 @@ class Partition(val topic: String, case Some(leaderReplica) => val replica = getReplica(replicaId).get val leaderHW = leaderReplica.highWatermark + val fetchOffset = logReadResult.info.fetchOffsetMetadata.messageOffset if (!inSyncReplicas.contains(replica) && assignedReplicas.map(_.brokerId).contains(replicaId) && - replica.logEndOffset.offsetDiff(leaderHW) >= 0) { + replica.logEndOffset.offsetDiff(leaderHW) >= 0 && + leaderEpochStartOffsetOpt.exists(fetchOffset >= _)) { val newInSyncReplicas = inSyncReplicas + replica info(s"Expanding ISR from ${inSyncReplicas.map(_.brokerId).mkString(",")} " + s"to ${newInSyncReplicas.map(_.brokerId).mkString(",")}") @@ -545,15 +599,48 @@ class Partition(val topic: String, laggingReplicas } - def appendRecordsToFutureReplica(records: MemoryRecords) { - getReplica(Request.FutureLocalReplicaId).get.log.get.appendAsFollower(records) + private def doAppendRecordsToFollowerOrFutureReplica(records: MemoryRecords, isFuture: Boolean): Unit = { + inReadLock(leaderIsrUpdateLock) { + if (isFuture) { + // The read lock is needed to handle race condition if request handler thread tries to + // remove future replica after receiving AlterReplicaLogDirsRequest. + inReadLock(leaderIsrUpdateLock) { + getReplica(Request.FutureLocalReplicaId) match { + case Some(replica) => replica.log.get.appendAsFollower(records) + case None => // Future replica is removed by a non-ReplicaAlterLogDirsThread before this method is called + } + } + } else { + // The read lock is needed to prevent the follower replica from being updated while ReplicaAlterDirThread + // is executing maybeDeleteAndSwapFutureReplica() to replace follower replica with the future replica. + getReplicaOrException().log.get.appendAsFollower(records) + } + } } - def appendRecordsToFollower(records: MemoryRecords) { - // The read lock is needed to prevent the follower replica from being updated while ReplicaAlterDirThread - // is executing maybeDeleteAndSwapFutureReplica() to replace follower replica with the future replica. - inReadLock(leaderIsrUpdateLock) { - getReplica().get.log.get.appendAsFollower(records) + def appendRecordsToFollowerOrFutureReplica(records: MemoryRecords, isFuture: Boolean) { + try { + doAppendRecordsToFollowerOrFutureReplica(records, isFuture) + } catch { + case e: UnexpectedAppendOffsetException => + val replica = if (isFuture) getReplicaOrException(Request.FutureLocalReplicaId) else getReplicaOrException() + val logEndOffset = replica.logEndOffset.messageOffset + if (logEndOffset == replica.logStartOffset && + e.firstOffset < logEndOffset && e.lastOffset >= logEndOffset) { + // This may happen if the log start offset on the leader (or current replica) falls in + // the middle of the batch due to delete records request and the follower tries to + // fetch its first offset from the leader. + // We handle this case here instead of Log#append() because we will need to remove the + // segment that start with log start offset and create a new one with earlier offset + // (base offset of the batch), which will move recoveryPoint backwards, so we will need + // to checkpoint the new recovery point before we append + val replicaName = if (isFuture) "future replica" else "follower" + info(s"Unexpected offset in append to $topicPartition. First offset ${e.firstOffset} is less than log start offset ${replica.logStartOffset}." + + s" Since this is the first record to be appended to the $replicaName's log, will start the log from offset ${e.firstOffset}.") + truncateFullyAndStartAt(e.firstOffset, isFuture) + doAppendRecordsToFollowerOrFutureReplica(records, isFuture) + } else + throw e } } @@ -657,7 +744,7 @@ class Partition(val topic: String, inReadLock(leaderIsrUpdateLock) { leaderReplicaIfLocal match { case Some(leaderReplica) => - val (epoch, offset) = leaderReplica.epochs.get.endOffsetFor(leaderEpoch) + val (epoch, offset) = leaderReplica.endOffsetFor(leaderEpoch) new EpochEndOffset(NONE, epoch, offset) case None => new EpochEndOffset(NOT_LEADER_FOR_PARTITION, UNDEFINED_EPOCH, UNDEFINED_EPOCH_OFFSET) diff --git a/core/src/main/scala/kafka/cluster/Replica.scala b/core/src/main/scala/kafka/cluster/Replica.scala index 4b65e439e2c74..7ddd58f6c9247 100644 --- a/core/src/main/scala/kafka/cluster/Replica.scala +++ b/core/src/main/scala/kafka/cluster/Replica.scala @@ -18,11 +18,13 @@ package kafka.cluster import kafka.log.Log +import kafka.server.epoch.LeaderEpochFileCache import kafka.utils.Logging import kafka.server.{LogOffsetMetadata, LogReadResult} -import kafka.common.KafkaException -import org.apache.kafka.common.TopicPartition +import org.apache.kafka.common.{KafkaException, TopicPartition} import org.apache.kafka.common.errors.OffsetOutOfRangeException +import org.apache.kafka.common.record.RecordVersion +import org.apache.kafka.common.requests.EpochEndOffset import org.apache.kafka.common.utils.Time class Replica(val brokerId: Int, @@ -55,7 +57,21 @@ class Replica(val brokerId: Int, def lastCaughtUpTimeMs = _lastCaughtUpTimeMs - val epochs = log.map(_.leaderEpochCache) + def epochs: Option[LeaderEpochFileCache] = { + log.flatMap { log => + if (log.supportsLeaderEpoch) + Some(log.leaderEpochCache) + else + None + } + } + + def endOffsetFor(requestedEpoch: Int): (Int, Long) = { + epochs match { + case Some(cache) => cache.endOffsetFor(requestedEpoch) + case None => (EpochEndOffset.UNDEFINED_EPOCH, EpochEndOffset.UNDEFINED_EPOCH_OFFSET) + } + } info(s"Replica loaded for partition $topicPartition with initial high watermark $initialHighWatermarkValue") log.foreach(_.onHighWatermarkIncremented(initialHighWatermarkValue)) diff --git a/core/src/main/scala/kafka/common/AuthorizationException.scala b/core/src/main/scala/kafka/common/AuthorizationException.scala deleted file mode 100644 index 55919a59896aa..0000000000000 --- a/core/src/main/scala/kafka/common/AuthorizationException.scala +++ /dev/null @@ -1,36 +0,0 @@ -/** - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package kafka.common - -/** - * Exception thrown when a principal is not authorized to perform an operation. - * @param message - */ -abstract class AuthorizationException(message: String) extends RuntimeException(message) { -} - -class TopicAuthorizationException(message: String) extends AuthorizationException(message) { - def this() = this(null) -} - -class GroupAuthorizationException(message: String) extends AuthorizationException(message) { - def this() = this(null) -} - -class ClusterAuthorizationException(message: String) extends AuthorizationException(message) { - def this() = this(null) -} diff --git a/core/src/main/scala/kafka/common/BrokerNotAvailableException.scala b/core/src/main/scala/kafka/common/BrokerNotAvailableException.scala deleted file mode 100644 index 611bed6f3181f..0000000000000 --- a/core/src/main/scala/kafka/common/BrokerNotAvailableException.scala +++ /dev/null @@ -1,22 +0,0 @@ -/** - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package kafka.common - -class BrokerNotAvailableException(message: String) extends RuntimeException(message) { - def this() = this(null) -} \ No newline at end of file diff --git a/core/src/main/scala/kafka/common/Config.scala b/core/src/main/scala/kafka/common/Config.scala index d24fb0df6352c..4110ba7b07315 100644 --- a/core/src/main/scala/kafka/common/Config.scala +++ b/core/src/main/scala/kafka/common/Config.scala @@ -19,6 +19,7 @@ package kafka.common import util.matching.Regex import kafka.utils.Logging +import org.apache.kafka.common.errors.InvalidConfigurationException trait Config extends Logging { @@ -29,8 +30,8 @@ trait Config extends Logging { rgx.findFirstIn(value) match { case Some(t) => if (!t.equals(value)) - throw new InvalidConfigException(prop + " " + value + " is illegal, contains a character other than ASCII alphanumerics, '.', '_' and '-'") - case None => throw new InvalidConfigException(prop + " " + value + " is illegal, contains a character other than ASCII alphanumerics, '.', '_' and '-'") + throw new InvalidConfigurationException(prop + " " + value + " is illegal, contains a character other than ASCII alphanumerics, '.', '_' and '-'") + case None => throw new InvalidConfigurationException(prop + " " + value + " is illegal, contains a character other than ASCII alphanumerics, '.', '_' and '-'") } } } diff --git a/core/src/main/scala/kafka/common/ConsumerCoordinatorNotAvailableException.scala b/core/src/main/scala/kafka/common/ConsumerCoordinatorNotAvailableException.scala deleted file mode 100644 index 8e02d264e9447..0000000000000 --- a/core/src/main/scala/kafka/common/ConsumerCoordinatorNotAvailableException.scala +++ /dev/null @@ -1,22 +0,0 @@ -/** - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package kafka.common - -class ConsumerCoordinatorNotAvailableException(message: String) extends RuntimeException(message) { - def this() = this(null) -} \ No newline at end of file diff --git a/core/src/main/scala/kafka/common/ControllerMovedException.scala b/core/src/main/scala/kafka/common/ControllerMovedException.scala deleted file mode 100644 index 39cf36d34059a..0000000000000 --- a/core/src/main/scala/kafka/common/ControllerMovedException.scala +++ /dev/null @@ -1,23 +0,0 @@ -/** - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. -*/ - -package kafka.common - -class ControllerMovedException(message: String, cause: Throwable) extends RuntimeException(message, cause) { - def this(message: String) = this(message, null) - def this() = this(null, null) -} \ No newline at end of file diff --git a/core/src/main/scala/kafka/common/ErrorMapping.scala b/core/src/main/scala/kafka/common/ErrorMapping.scala deleted file mode 100644 index 9f290731a7075..0000000000000 --- a/core/src/main/scala/kafka/common/ErrorMapping.scala +++ /dev/null @@ -1,117 +0,0 @@ -/** - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package kafka.common - -import java.nio.ByteBuffer - -import kafka.message.InvalidMessageException -import org.apache.kafka.common.errors.InvalidTopicException - -import scala.Predef._ - -/** - * A bi-directional mapping between error codes and exceptions - */ -object ErrorMapping { - val EmptyByteBuffer = ByteBuffer.allocate(0) - - val UnknownCode: Short = -1 - val NoError: Short = 0 - val OffsetOutOfRangeCode: Short = 1 - val InvalidMessageCode: Short = 2 - val UnknownTopicOrPartitionCode: Short = 3 - val InvalidFetchSizeCode: Short = 4 - val LeaderNotAvailableCode: Short = 5 - val NotLeaderForPartitionCode: Short = 6 - val RequestTimedOutCode: Short = 7 - val BrokerNotAvailableCode: Short = 8 - val ReplicaNotAvailableCode: Short = 9 - val MessageSizeTooLargeCode: Short = 10 - val StaleControllerEpochCode: Short = 11 - val OffsetMetadataTooLargeCode: Short = 12 - val StaleLeaderEpochCode: Short = 13 - val OffsetsLoadInProgressCode: Short = 14 - val ConsumerCoordinatorNotAvailableCode: Short = 15 - val NotCoordinatorForConsumerCode: Short = 16 - val InvalidTopicCode: Short = 17 - val MessageSetSizeTooLargeCode: Short = 18 - val NotEnoughReplicasCode: Short = 19 - val NotEnoughReplicasAfterAppendCode: Short = 20 - // 21: InvalidRequiredAcks - // 22: IllegalConsumerGeneration - // 23: INCONSISTENT_PARTITION_ASSIGNMENT_STRATEGY - // 24: UNKNOWN_PARTITION_ASSIGNMENT_STRATEGY - // 25: UNKNOWN_CONSUMER_ID - // 26: INVALID_SESSION_TIMEOUT - // 27: REBALANCE_IN_PROGRESS - // 28: INVALID_COMMIT_OFFSET_SIZE - val TopicAuthorizationCode: Short = 29 - val GroupAuthorizationCode: Short = 30 - val ClusterAuthorizationCode: Short = 31 - // 32: INVALID_TIMESTAMP - // 33: UNSUPPORTED_SASL_MECHANISM - // 34: ILLEGAL_SASL_STATE - // 35: UNSUPPORTED_VERSION - // 36: TOPIC_ALREADY_EXISTS - // 37: INVALID_PARTITIONS - // 38: INVALID_REPLICATION_FACTOR - // 39: INVALID_REPLICA_ASSIGNMENT - // 40: INVALID_CONFIG - // 41: NOT_CONTROLLER - // 42: INVALID_REQUEST - - private val exceptionToCode = - Map[Class[Throwable], Short]( - classOf[OffsetOutOfRangeException].asInstanceOf[Class[Throwable]] -> OffsetOutOfRangeCode, - classOf[InvalidMessageException].asInstanceOf[Class[Throwable]] -> InvalidMessageCode, - classOf[UnknownTopicOrPartitionException].asInstanceOf[Class[Throwable]] -> UnknownTopicOrPartitionCode, - classOf[InvalidMessageSizeException].asInstanceOf[Class[Throwable]] -> InvalidFetchSizeCode, - classOf[LeaderNotAvailableException].asInstanceOf[Class[Throwable]] -> LeaderNotAvailableCode, - classOf[NotLeaderForPartitionException].asInstanceOf[Class[Throwable]] -> NotLeaderForPartitionCode, - classOf[RequestTimedOutException].asInstanceOf[Class[Throwable]] -> RequestTimedOutCode, - classOf[BrokerNotAvailableException].asInstanceOf[Class[Throwable]] -> BrokerNotAvailableCode, - classOf[ReplicaNotAvailableException].asInstanceOf[Class[Throwable]] -> ReplicaNotAvailableCode, - classOf[MessageSizeTooLargeException].asInstanceOf[Class[Throwable]] -> MessageSizeTooLargeCode, - classOf[ControllerMovedException].asInstanceOf[Class[Throwable]] -> StaleControllerEpochCode, - classOf[OffsetMetadataTooLargeException].asInstanceOf[Class[Throwable]] -> OffsetMetadataTooLargeCode, - classOf[OffsetsLoadInProgressException].asInstanceOf[Class[Throwable]] -> OffsetsLoadInProgressCode, - classOf[ConsumerCoordinatorNotAvailableException].asInstanceOf[Class[Throwable]] -> ConsumerCoordinatorNotAvailableCode, - classOf[NotCoordinatorForConsumerException].asInstanceOf[Class[Throwable]] -> NotCoordinatorForConsumerCode, - classOf[InvalidTopicException].asInstanceOf[Class[Throwable]] -> InvalidTopicCode, - classOf[MessageSetSizeTooLargeException].asInstanceOf[Class[Throwable]] -> MessageSetSizeTooLargeCode, - classOf[NotEnoughReplicasException].asInstanceOf[Class[Throwable]] -> NotEnoughReplicasCode, - classOf[NotEnoughReplicasAfterAppendException].asInstanceOf[Class[Throwable]] -> NotEnoughReplicasAfterAppendCode, - classOf[TopicAuthorizationException].asInstanceOf[Class[Throwable]] -> TopicAuthorizationCode, - classOf[GroupAuthorizationException].asInstanceOf[Class[Throwable]] -> GroupAuthorizationCode, - classOf[ClusterAuthorizationException].asInstanceOf[Class[Throwable]] -> ClusterAuthorizationCode - ).withDefaultValue(UnknownCode) - - /* invert the mapping */ - 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/FailedToSendMessageException.scala b/core/src/main/scala/kafka/common/FailedToSendMessageException.scala deleted file mode 100644 index 6aee4142d843a..0000000000000 --- a/core/src/main/scala/kafka/common/FailedToSendMessageException.scala +++ /dev/null @@ -1,23 +0,0 @@ -/** - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. -*/ -package kafka.common - -/** - * Indicates a producer pool initialization problem -*/ -class FailedToSendMessageException(message: String, t: Throwable) extends RuntimeException(message, t) { -} diff --git a/core/src/main/scala/kafka/common/LeaderNotAvailableException.scala b/core/src/main/scala/kafka/common/IndexOffsetOverflowException.scala similarity index 78% rename from core/src/main/scala/kafka/common/LeaderNotAvailableException.scala rename to core/src/main/scala/kafka/common/IndexOffsetOverflowException.scala index 972728edb0951..5dd9b43e9e843 100644 --- a/core/src/main/scala/kafka/common/LeaderNotAvailableException.scala +++ b/core/src/main/scala/kafka/common/IndexOffsetOverflowException.scala @@ -18,9 +18,8 @@ package kafka.common /** - * Thrown when a request is made for partition, but no leader exists for that partition + * Indicates that an attempt was made to append a message whose offset could cause the index offset to overflow. */ -class LeaderNotAvailableException(message: String, cause: Throwable) extends RuntimeException(message, cause) { +class IndexOffsetOverflowException(message: String, cause: Throwable) extends org.apache.kafka.common.KafkaException(message, cause) { def this(message: String) = this(message, null) - def this() = this(null, null) } diff --git a/core/src/main/scala/kafka/common/InterBrokerSendThread.scala b/core/src/main/scala/kafka/common/InterBrokerSendThread.scala index 60635f17073d2..aedaac79d2774 100644 --- a/core/src/main/scala/kafka/common/InterBrokerSendThread.scala +++ b/core/src/main/scala/kafka/common/InterBrokerSendThread.scala @@ -39,7 +39,7 @@ abstract class InterBrokerSendThread(name: String, extends ShutdownableThread(name, isInterruptible) { def generateRequests(): Iterable[RequestAndCompletionHandler] - def unsentExpiryMs: Int + def requestTimeoutMs: Int private val unsentRequests = new UnsentRequests def hasUnsentRequests = unsentRequests.iterator().hasNext @@ -57,7 +57,8 @@ abstract class InterBrokerSendThread(name: String, generateRequests().foreach { request => val completionHandler = request.handler unsentRequests.put(request.destination, - networkClient.newClientRequest(request.destination.idString, request.request, now, true, completionHandler)) + networkClient.newClientRequest(request.destination.idString, request.request, now, true, + requestTimeoutMs, completionHandler)) } try { @@ -118,9 +119,9 @@ abstract class InterBrokerSendThread(name: String, private def failExpiredRequests(now: Long): Unit = { // clear all expired unsent requests - val expiredRequests = unsentRequests.removeExpiredRequests(now, unsentExpiryMs) - for (request <- expiredRequests.asScala) { - debug(s"Failed to send the following request after $unsentExpiryMs ms: $request") + val timedOutRequests = unsentRequests.removeAllTimedOut(now) + for (request <- timedOutRequests.asScala) { + debug(s"Failed to send the following request after ${request.requestTimeoutMs} ms: $request") completeWithDisconnect(request, now, null) } } @@ -152,14 +153,15 @@ private class UnsentRequests { requests.add(request) } - def removeExpiredRequests(now: Long, unsentExpiryMs: Long): Collection[ClientRequest] = { + def removeAllTimedOut(now: Long): Collection[ClientRequest] = { val expiredRequests = new ArrayList[ClientRequest] for (requests <- unsent.values.asScala) { val requestIterator = requests.iterator var foundExpiredRequest = false while (requestIterator.hasNext && !foundExpiredRequest) { val request = requestIterator.next - if (request.createdTimeMs < now - unsentExpiryMs) { + val elapsedMs = Math.max(0, now - request.createdTimeMs) + if (elapsedMs > request.requestTimeoutMs) { expiredRequests.add(request) requestIterator.remove() foundExpiredRequest = true diff --git a/core/src/main/scala/kafka/common/InvalidConfigException.scala b/core/src/main/scala/kafka/common/InvalidConfigException.scala deleted file mode 100644 index 643784638deb1..0000000000000 --- a/core/src/main/scala/kafka/common/InvalidConfigException.scala +++ /dev/null @@ -1,25 +0,0 @@ -/** - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. -*/ - -package kafka.common - -/** - * Indicates that the given config parameter has invalid value - */ -class InvalidConfigException(message: String) extends RuntimeException(message) { - def this() = this(null) -} diff --git a/core/src/main/scala/kafka/common/InvalidOffsetException.scala b/core/src/main/scala/kafka/common/InvalidOffsetException.scala deleted file mode 100644 index c6811d7c0271e..0000000000000 --- a/core/src/main/scala/kafka/common/InvalidOffsetException.scala +++ /dev/null @@ -1,22 +0,0 @@ -/** - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package kafka.common - -class InvalidOffsetException(message: String) extends RuntimeException(message) { - def this() = this(null) -} \ No newline at end of file diff --git a/core/src/main/scala/kafka/common/KafkaException.scala b/core/src/main/scala/kafka/common/KafkaException.scala index e72d151c09e97..61b3ba316e0e7 100644 --- a/core/src/main/scala/kafka/common/KafkaException.scala +++ b/core/src/main/scala/kafka/common/KafkaException.scala @@ -17,7 +17,10 @@ package kafka.common /** - * Generic Kafka exception + * Usage of this class is discouraged. Use org.apache.kafka.common.KafkaException instead. + * + * This class will be removed once ZkUtils and the kafka.security.auth classes are removed. + * The former is internal, but widely used, so we are leaving it in the codebase for now. */ class KafkaException(message: String, t: Throwable) extends RuntimeException(message, t) { def this(message: String) = this(message, null) diff --git a/core/src/main/scala/kafka/common/KafkaStorageException.scala b/core/src/main/scala/kafka/common/KafkaStorageException.scala deleted file mode 100644 index e0ecff3ec45aa..0000000000000 --- a/core/src/main/scala/kafka/common/KafkaStorageException.scala +++ /dev/null @@ -1,27 +0,0 @@ -/** - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. -*/ -package kafka.common - -/** - * Kafka exception caused by disk-related IOException - * This class is deprecated and will be replaced by org.apache.kafka.common.errors.KafkaStorageException - */ -@Deprecated -class KafkaStorageException(message: String, t: Throwable) extends RuntimeException(message, t) { - def this(message: String) = this(message, null) - def this(t: Throwable) = this("", t) -} diff --git a/core/src/main/scala/kafka/common/ConsumerRebalanceFailedException.scala b/core/src/main/scala/kafka/common/LogSegmentOffsetOverflowException.scala similarity index 56% rename from core/src/main/scala/kafka/common/ConsumerRebalanceFailedException.scala rename to core/src/main/scala/kafka/common/LogSegmentOffsetOverflowException.scala index 2f4c2cf70220a..2de5906109af9 100644 --- a/core/src/main/scala/kafka/common/ConsumerRebalanceFailedException.scala +++ b/core/src/main/scala/kafka/common/LogSegmentOffsetOverflowException.scala @@ -17,10 +17,14 @@ package kafka.common +import kafka.log.LogSegment + /** - * Thrown when a request is made for broker but no brokers with that topic - * exist. + * Indicates that the log segment contains one or more messages that overflow the offset (and / or time) index. This is + * not a typical scenario, and could only happen when brokers have log segments that were created before the patch for + * KAFKA-5413. With KAFKA-6264, we have the ability to split such log segments into multiple log segments such that we + * do not have any segments with offset overflow. */ -class ConsumerRebalanceFailedException(message: String) extends RuntimeException(message) { - def this() = this(null) +class LogSegmentOffsetOverflowException(val segment: LogSegment, val offset: Long) + extends org.apache.kafka.common.KafkaException(s"Detected offset overflow at offset $offset in segment $segment") { } diff --git a/core/src/main/scala/kafka/common/MessageSetSizeTooLargeException.scala b/core/src/main/scala/kafka/common/MessageSetSizeTooLargeException.scala deleted file mode 100644 index 94a616ed3972a..0000000000000 --- a/core/src/main/scala/kafka/common/MessageSetSizeTooLargeException.scala +++ /dev/null @@ -1,22 +0,0 @@ -/** - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package kafka.common - -class MessageSetSizeTooLargeException(message: String) extends RuntimeException(message) { - def this() = this(null) -} diff --git a/core/src/main/scala/kafka/common/MessageSizeTooLargeException.scala b/core/src/main/scala/kafka/common/MessageSizeTooLargeException.scala deleted file mode 100644 index 2d18324fb7d2e..0000000000000 --- a/core/src/main/scala/kafka/common/MessageSizeTooLargeException.scala +++ /dev/null @@ -1,22 +0,0 @@ -/** - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package kafka.common - -class MessageSizeTooLargeException(message: String) extends RuntimeException(message) { - def this() = this(null) -} \ No newline at end of file diff --git a/core/src/main/scala/kafka/common/MessageStreamsExistException.scala b/core/src/main/scala/kafka/common/MessageStreamsExistException.scala deleted file mode 100644 index b904ed04d7387..0000000000000 --- a/core/src/main/scala/kafka/common/MessageStreamsExistException.scala +++ /dev/null @@ -1,23 +0,0 @@ -/** - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. -*/ -package kafka.common - -/** - * Indicates a createMessageStreams can't be called more than once -*/ -class MessageStreamsExistException(message: String, t: Throwable) extends RuntimeException(message, t) { -} diff --git a/core/src/main/scala/kafka/common/NoBrokersForPartitionException.scala b/core/src/main/scala/kafka/common/NoBrokersForPartitionException.scala deleted file mode 100644 index 4577b298032ab..0000000000000 --- a/core/src/main/scala/kafka/common/NoBrokersForPartitionException.scala +++ /dev/null @@ -1,26 +0,0 @@ -/** - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package kafka.common - -/** - * Thrown when a request is made for broker but no brokers with that topic - * exist. - */ -class NoBrokersForPartitionException(message: String) extends RuntimeException(message) { - def this() = this(null) -} \ No newline at end of file diff --git a/core/src/main/scala/kafka/common/NotCoordinatorForConsumerException.scala b/core/src/main/scala/kafka/common/NotCoordinatorForConsumerException.scala deleted file mode 100644 index 1eb74be038eaa..0000000000000 --- a/core/src/main/scala/kafka/common/NotCoordinatorForConsumerException.scala +++ /dev/null @@ -1,22 +0,0 @@ -/** - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package kafka.common - -class NotCoordinatorForConsumerException(message: String) extends RuntimeException(message) { - def this() = this(null) -} \ No newline at end of file diff --git a/core/src/main/scala/kafka/common/NotEnoughReplicasAfterAppendException.scala b/core/src/main/scala/kafka/common/NotEnoughReplicasAfterAppendException.scala deleted file mode 100644 index c4f9def6162e9..0000000000000 --- a/core/src/main/scala/kafka/common/NotEnoughReplicasAfterAppendException.scala +++ /dev/null @@ -1,27 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package kafka.common - -/** - * 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 deleted file mode 100644 index bfbe0ee4a5a15..0000000000000 --- a/core/src/main/scala/kafka/common/NotEnoughReplicasException.scala +++ /dev/null @@ -1,25 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package kafka.common - -/** - * 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/NotLeaderForPartitionException.scala b/core/src/main/scala/kafka/common/NotLeaderForPartitionException.scala deleted file mode 100644 index b4558f89f0a23..0000000000000 --- a/core/src/main/scala/kafka/common/NotLeaderForPartitionException.scala +++ /dev/null @@ -1,25 +0,0 @@ -/** - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package kafka.common - -/** - * Thrown when a request is made for partition on a broker that is NOT a leader for that partition - */ -class NotLeaderForPartitionException(message: String) extends RuntimeException(message) { - def this() = this(null) -} \ No newline at end of file diff --git a/core/src/main/scala/kafka/common/OffsetMetadataTooLargeException.scala b/core/src/main/scala/kafka/common/OffsetMetadataTooLargeException.scala deleted file mode 100644 index 50edb273b3a79..0000000000000 --- a/core/src/main/scala/kafka/common/OffsetMetadataTooLargeException.scala +++ /dev/null @@ -1,27 +0,0 @@ -/** - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package kafka.common - -/** - * Indicates the client has specified offset metadata that exceeds the configured - * maximum size in bytes - */ -class OffsetMetadataTooLargeException(message: String) extends RuntimeException(message) { - def this() = this(null) -} - diff --git a/core/src/main/scala/kafka/common/OffsetOutOfRangeException.scala b/core/src/main/scala/kafka/common/OffsetOutOfRangeException.scala deleted file mode 100644 index 0a2514cc0d99f..0000000000000 --- a/core/src/main/scala/kafka/common/OffsetOutOfRangeException.scala +++ /dev/null @@ -1,26 +0,0 @@ -/** - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package kafka.common - -/** - * Indicates the client has requested a range no longer available on the server - */ -class OffsetOutOfRangeException(message: String) extends RuntimeException(message) { - def this() = this(null) -} - diff --git a/core/src/main/scala/kafka/common/UnknownException.scala b/core/src/main/scala/kafka/common/OffsetsOutOfOrderException.scala similarity index 82% rename from core/src/main/scala/kafka/common/UnknownException.scala rename to core/src/main/scala/kafka/common/OffsetsOutOfOrderException.scala index 6cf0fc9db316b..f8daaa4a181b2 100644 --- a/core/src/main/scala/kafka/common/UnknownException.scala +++ b/core/src/main/scala/kafka/common/OffsetsOutOfOrderException.scala @@ -18,6 +18,8 @@ package kafka.common /** - * If we don't know what else it is, call it this + * Indicates the follower received records with non-monotonically increasing offsets */ -class UnknownException extends RuntimeException +class OffsetsOutOfOrderException(message: String) extends RuntimeException(message) { +} + diff --git a/core/src/main/scala/kafka/common/QueueFullException.scala b/core/src/main/scala/kafka/common/QueueFullException.scala deleted file mode 100644 index 27c04821ef54d..0000000000000 --- a/core/src/main/scala/kafka/common/QueueFullException.scala +++ /dev/null @@ -1,23 +0,0 @@ -package kafka.common - -/** - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -/* Indicates the queue for sending messages is full of unsent messages */ -class QueueFullException(message: String) extends RuntimeException(message) { - def this() = this(null) -} diff --git a/core/src/main/scala/kafka/common/ReplicaNotAvailableException.scala b/core/src/main/scala/kafka/common/ReplicaNotAvailableException.scala deleted file mode 100644 index f1d1eadc546ec..0000000000000 --- a/core/src/main/scala/kafka/common/ReplicaNotAvailableException.scala +++ /dev/null @@ -1,26 +0,0 @@ -/** - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package kafka.common - -/** - * Thrown when a request is made for partition, but no leader exists for that partition - */ -class ReplicaNotAvailableException(cause: Throwable, message: String = "") extends RuntimeException(cause) { - def this() = this(null, "") - def this(message: String) = this(null, message) -} \ No newline at end of file diff --git a/core/src/main/scala/kafka/common/RequestTimedOutException.scala b/core/src/main/scala/kafka/common/RequestTimedOutException.scala deleted file mode 100644 index faedea8b40b5c..0000000000000 --- a/core/src/main/scala/kafka/common/RequestTimedOutException.scala +++ /dev/null @@ -1,29 +0,0 @@ -/** - * - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package kafka.common - - -/** - * Thrown when a produce request times out - i.e., if one or more partitions it - * sends messages to receives fewer than the requiredAcks that is specified in - * the produce request. - */ -class RequestTimedOutException(message: String) extends RuntimeException(message) { - def this() = this(null) -} \ No newline at end of file diff --git a/core/src/main/scala/kafka/common/StreamEndException.scala b/core/src/main/scala/kafka/common/StreamEndException.scala deleted file mode 100644 index a9410bcb907b3..0000000000000 --- a/core/src/main/scala/kafka/common/StreamEndException.scala +++ /dev/null @@ -1,23 +0,0 @@ -/** - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package kafka.common - -/** - * An exception that indicates KafkaStream has ended. - */ -class StreamEndException() extends RuntimeException { -} diff --git a/core/src/main/scala/kafka/common/UnavailableProducerException.scala b/core/src/main/scala/kafka/common/UnavailableProducerException.scala deleted file mode 100644 index 885c98df63f3b..0000000000000 --- a/core/src/main/scala/kafka/common/UnavailableProducerException.scala +++ /dev/null @@ -1,24 +0,0 @@ -/** - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. -*/ -package kafka.common - -/** - * Indicates a producer pool initialization problem -*/ -class UnavailableProducerException(message: String) extends RuntimeException(message) { - def this() = this(null) -} diff --git a/core/src/main/scala/kafka/common/OffsetsLoadInProgressException.scala b/core/src/main/scala/kafka/common/UnexpectedAppendOffsetException.scala similarity index 62% rename from core/src/main/scala/kafka/common/OffsetsLoadInProgressException.scala rename to core/src/main/scala/kafka/common/UnexpectedAppendOffsetException.scala index 1c8e96eefc7f0..e719a93006d31 100644 --- a/core/src/main/scala/kafka/common/OffsetsLoadInProgressException.scala +++ b/core/src/main/scala/kafka/common/UnexpectedAppendOffsetException.scala @@ -18,9 +18,12 @@ package kafka.common /** - * Indicates that offsets are currently being loaded from disk into the cache so offset fetch requests cannot be satisfied. + * Indicates the follower or the future replica received records from the leader (or current + * replica) with first offset less than expected next offset. + * @param firstOffset The first offset of the records to append + * @param lastOffset The last offset of the records to append */ -class OffsetsLoadInProgressException(message: String) extends RuntimeException(message) { - def this() = this(null) +class UnexpectedAppendOffsetException(val message: String, + val firstOffset: Long, + val lastOffset: Long) extends RuntimeException(message) { } - diff --git a/core/src/main/scala/kafka/common/UnknownMagicByteException.scala b/core/src/main/scala/kafka/common/UnknownMagicByteException.scala deleted file mode 100644 index 544d42687ba8e..0000000000000 --- a/core/src/main/scala/kafka/common/UnknownMagicByteException.scala +++ /dev/null @@ -1,26 +0,0 @@ -/** - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package kafka.common - -/** - * Indicates the client has requested a range no longer available on the server - */ -class UnknownMagicByteException(message: String) extends RuntimeException(message) { - def this() = this(null) -} - diff --git a/core/src/main/scala/kafka/common/UnknownTopicOrPartitionException.scala b/core/src/main/scala/kafka/common/UnknownTopicOrPartitionException.scala deleted file mode 100644 index f382d16de9c39..0000000000000 --- a/core/src/main/scala/kafka/common/UnknownTopicOrPartitionException.scala +++ /dev/null @@ -1,26 +0,0 @@ -/** - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. -*/ -package kafka.common - -/** - * 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 index 51798519cb137..65c350632b2ab 100644 --- a/core/src/main/scala/kafka/common/ZkNodeChangeNotificationListener.scala +++ b/core/src/main/scala/kafka/common/ZkNodeChangeNotificationListener.scala @@ -16,6 +16,7 @@ */ package kafka.common +import java.nio.charset.StandardCharsets.UTF_8 import java.util.concurrent.LinkedBlockingQueue import java.util.concurrent.atomic.AtomicBoolean @@ -24,6 +25,8 @@ import kafka.zk.{KafkaZkClient, StateChangeHandlers} import kafka.zookeeper.{StateChangeHandler, ZNodeChildChangeHandler} import org.apache.kafka.common.utils.Time +import scala.util.{Failure, Try} + /** * Handle the notificationMessage. */ @@ -83,12 +86,7 @@ class ZkNodeChangeNotificationListener(private val zkClient: KafkaZkClient, for (notification <- notifications) { val changeId = changeNumber(notification) if (changeId > lastExecutedChange) { - val changeZnode = seqNodeRoot + "/" + notification - val (data, _) = zkClient.getDataAndStat(changeZnode) - data match { - case Some(d) => notificationHandler.processNotification(d) - case None => warn(s"read null data from $changeZnode when processing notification $notification") - } + processNotification(notification) lastExecutedChange = changeId } } @@ -100,6 +98,18 @@ class ZkNodeChangeNotificationListener(private val zkClient: KafkaZkClient, } } + private def processNotification(notification: String): Unit = { + val changeZnode = seqNodeRoot + "/" + notification + val (data, _) = zkClient.getDataAndStat(changeZnode) + data match { + case Some(d) => Try(notificationHandler.processNotification(d)) match { + case Failure(e) => error(s"error processing change notification ${new String(d, UTF_8)} from $changeZnode", e) + case _ => + } + case None => warn(s"read null data from $changeZnode") + } + } + private def addChangeNotification(): Unit = { if (!isClosed.get && queue.peek() == null) queue.put(new ChangeNotification) diff --git a/core/src/main/scala/kafka/consumer/BaseConsumer.scala b/core/src/main/scala/kafka/consumer/BaseConsumer.scala deleted file mode 100644 index 9e49fe4dc8b39..0000000000000 --- a/core/src/main/scala/kafka/consumer/BaseConsumer.scala +++ /dev/null @@ -1,172 +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.consumer - -import java.util.{Collections, Properties} -import java.util.regex.Pattern - -import kafka.api.OffsetRequest -import kafka.common.StreamEndException -import kafka.message.Message -import org.apache.kafka.clients.consumer.Consumer -import org.apache.kafka.common.record.TimestampType -import org.apache.kafka.common.TopicPartition -import org.apache.kafka.common.header.Headers -import org.apache.kafka.common.header.internals.RecordHeaders - -/** - * A base consumer used to abstract both old and new consumer - * this class should be removed (along with BaseProducer) - * once we deprecate old consumer - */ -@deprecated("This trait has been deprecated and will be removed in a future release. " + - "Please use org.apache.kafka.clients.consumer.KafkaConsumer instead.", "0.11.0.0") -trait BaseConsumer { - def receive(): BaseConsumerRecord - def stop() - def cleanup() - def commit() -} - -@deprecated("This class has been deprecated and will be removed in a future release. " + - "Please use org.apache.kafka.clients.consumer.ConsumerRecord instead.", "0.11.0.0") -case class BaseConsumerRecord(topic: String, - partition: Int, - offset: Long, - timestamp: Long = Message.NoTimestamp, - timestampType: TimestampType = TimestampType.NO_TIMESTAMP_TYPE, - key: Array[Byte], - value: Array[Byte], - headers: Headers = new RecordHeaders()) - -@deprecated("This class has been deprecated and will be removed in a future release. " + - "Please use org.apache.kafka.clients.consumer.KafkaConsumer instead.", "0.11.0.0") -class NewShinyConsumer(topic: Option[String], partitionId: Option[Int], offset: Option[Long], whitelist: Option[String], - consumer: Consumer[Array[Byte], Array[Byte]], val timeoutMs: Long = Long.MaxValue) extends BaseConsumer { - consumerInit() - var recordIter = consumer.poll(0).iterator - - def consumerInit() { - (topic, partitionId, offset, whitelist) match { - case (Some(topic), Some(partitionId), Some(offset), None) => - seek(topic, partitionId, offset) - case (Some(topic), Some(partitionId), None, None) => - // default to latest if no offset is provided - seek(topic, partitionId, OffsetRequest.LatestTime) - case (Some(topic), None, None, None) => - consumer.subscribe(Collections.singletonList(topic)) - case (None, None, None, Some(whitelist)) => - consumer.subscribe(Pattern.compile(whitelist)) - case _ => - throw new IllegalArgumentException("An invalid combination of arguments is provided. " + - "Exactly one of 'topic' or 'whitelist' must be provided. " + - "If 'topic' is provided, an optional 'partition' may also be provided. " + - "If 'partition' is provided, an optional 'offset' may also be provided, otherwise, consumption starts from the end of the partition.") - } - } - - def seek(topic: String, partitionId: Int, offset: Long) { - val topicPartition = new TopicPartition(topic, partitionId) - consumer.assign(Collections.singletonList(topicPartition)) - offset match { - case OffsetRequest.EarliestTime => consumer.seekToBeginning(Collections.singletonList(topicPartition)) - case OffsetRequest.LatestTime => consumer.seekToEnd(Collections.singletonList(topicPartition)) - case _ => consumer.seek(topicPartition, offset) - } - } - - def resetUnconsumedOffsets() { - val smallestUnconsumedOffsets = collection.mutable.Map[TopicPartition, Long]() - while (recordIter.hasNext) { - val record = recordIter.next() - val tp = new TopicPartition(record.topic, record.partition) - // avoid auto-committing offsets which haven't been consumed - smallestUnconsumedOffsets.getOrElseUpdate(tp, record.offset) - } - smallestUnconsumedOffsets.foreach { case (tp, offset) => consumer.seek(tp, offset) } - } - - override def receive(): BaseConsumerRecord = { - if (!recordIter.hasNext) { - recordIter = consumer.poll(timeoutMs).iterator - if (!recordIter.hasNext) - throw new ConsumerTimeoutException - } - - val record = recordIter.next - BaseConsumerRecord(record.topic, - record.partition, - record.offset, - record.timestamp, - record.timestampType, - record.key, - record.value, - record.headers) - } - - override def stop() { - this.consumer.wakeup() - } - - override def cleanup() { - resetUnconsumedOffsets() - this.consumer.close() - } - - override def commit() { - this.consumer.commitSync() - } -} - -@deprecated("This class has been deprecated and will be removed in a future release. " + - "Please use org.apache.kafka.clients.consumer.KafkaConsumer instead.", "0.11.0.0") -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 = { - if (!iter.hasNext()) - throw new StreamEndException - - val messageAndMetadata = iter.next - BaseConsumerRecord(messageAndMetadata.topic, - messageAndMetadata.partition, - messageAndMetadata.offset, - messageAndMetadata.timestamp, - messageAndMetadata.timestampType, - messageAndMetadata.key, - messageAndMetadata.message, - new RecordHeaders()) - } - - override def stop() { - this.consumerConnector.shutdown() - } - - override def cleanup() { - this.consumerConnector.shutdown() - } - - override def commit() { - this.consumerConnector.commitOffsets - } -} diff --git a/core/src/main/scala/kafka/consumer/FetchedDataChunk.scala b/core/src/main/scala/kafka/consumer/BaseConsumerRecord.scala similarity index 54% rename from core/src/main/scala/kafka/consumer/FetchedDataChunk.scala rename to core/src/main/scala/kafka/consumer/BaseConsumerRecord.scala index 91eb874150048..7628b6b65168c 100644 --- a/core/src/main/scala/kafka/consumer/FetchedDataChunk.scala +++ b/core/src/main/scala/kafka/consumer/BaseConsumerRecord.scala @@ -17,9 +17,17 @@ package kafka.consumer -import kafka.message.ByteBufferMessageSet +import org.apache.kafka.common.header.Headers +import org.apache.kafka.common.header.internals.RecordHeaders +import org.apache.kafka.common.record.{RecordBatch, TimestampType} -@deprecated("This class has been deprecated and will be removed in a future release.", "0.11.0.0") -case class FetchedDataChunk(messages: ByteBufferMessageSet, - topicInfo: PartitionTopicInfo, - fetchOffset: Long) +@deprecated("This class has been deprecated and will be removed in a future release. " + + "Please use org.apache.kafka.clients.consumer.ConsumerRecord instead.", "0.11.0.0") +case class BaseConsumerRecord(topic: String, + partition: Int, + offset: Long, + timestamp: Long = RecordBatch.NO_TIMESTAMP, + timestampType: TimestampType = TimestampType.NO_TIMESTAMP_TYPE, + key: Array[Byte], + value: Array[Byte], + headers: Headers = new RecordHeaders()) diff --git a/core/src/main/scala/kafka/consumer/ConsumerConfig.scala b/core/src/main/scala/kafka/consumer/ConsumerConfig.scala deleted file mode 100644 index bea0307389a1d..0000000000000 --- a/core/src/main/scala/kafka/consumer/ConsumerConfig.scala +++ /dev/null @@ -1,205 +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.consumer - -import java.util.Properties -import kafka.api.OffsetRequest -import kafka.utils._ -import kafka.common.{InvalidConfigException, Config} -import java.util.Locale - -@deprecated("This object has been deprecated and will be removed in a future release. " + - "Please use org.apache.kafka.clients.consumer.ConsumerConfig instead.", "0.11.0.0") -object ConsumerConfig extends Config { - val RefreshMetadataBackoffMs = 200 - val SocketTimeout = 30 * 1000 - val SocketBufferSize = 64*1024 - val FetchSize = 1024 * 1024 - val MaxFetchSize = 10*FetchSize - val NumConsumerFetchers = 1 - val DefaultFetcherBackoffMs = 1000 - val AutoCommit = true - val AutoCommitInterval = 60 * 1000 - val MaxQueuedChunks = 2 - val MaxRebalanceRetries = 4 - val AutoOffsetReset = OffsetRequest.LargestTimeString - val ConsumerTimeoutMs = -1 - val MinFetchBytes = 1 - val MaxFetchBytes = 50 * 1024 * 1024 - val MaxFetchWaitMs = 100 - val MirrorTopicsWhitelist = "" - val MirrorTopicsBlacklist = "" - val MirrorConsumerNumThreads = 1 - val OffsetsChannelBackoffMs = 1000 - val OffsetsChannelSocketTimeoutMs = 10000 - val OffsetsCommitMaxRetries = 5 - val OffsetsStorage = "zookeeper" - - val MirrorTopicsWhitelistProp = "mirror.topics.whitelist" - val MirrorTopicsBlacklistProp = "mirror.topics.blacklist" - val ExcludeInternalTopics = true - val DefaultPartitionAssignmentStrategy = "range" /* select between "range", and "roundrobin" */ - val MirrorConsumerNumThreadsProp = "mirror.consumer.numthreads" - val DefaultClientId = "" - - def validate(config: ConsumerConfig) { - validateClientId(config.clientId) - validateGroupId(config.groupId) - validateAutoOffsetReset(config.autoOffsetReset) - validateOffsetsStorage(config.offsetsStorage) - validatePartitionAssignmentStrategy(config.partitionAssignmentStrategy) - } - - def validateClientId(clientId: String) { - validateChars("client.id", clientId) - } - - def validateGroupId(groupId: String) { - validateChars("group.id", groupId) - } - - def validateAutoOffsetReset(autoOffsetReset: String) { - autoOffsetReset match { - case OffsetRequest.SmallestTimeString => - case OffsetRequest.LargestTimeString => - case _ => throw new InvalidConfigException("Wrong value " + autoOffsetReset + " of auto.offset.reset in ConsumerConfig; " + - "Valid values are " + OffsetRequest.SmallestTimeString + " and " + OffsetRequest.LargestTimeString) - } - } - - def validateOffsetsStorage(storage: String) { - storage match { - case "zookeeper" => - case "kafka" => - case _ => throw new InvalidConfigException("Wrong value " + storage + " of offsets.storage in consumer config; " + - "Valid values are 'zookeeper' and 'kafka'") - } - } - - def validatePartitionAssignmentStrategy(strategy: String) { - strategy match { - case "range" => - case "roundrobin" => - case _ => throw new InvalidConfigException("Wrong value " + strategy + " of partition.assignment.strategy in consumer config; " + - "Valid values are 'range' and 'roundrobin'") - } - } -} - -@deprecated("This class has been deprecated and will be removed in a future release. " + - "Please use org.apache.kafka.clients.consumer.ConsumerConfig instead.", "0.11.0.0") -class ConsumerConfig private (val props: VerifiableProperties) extends ZKConfig(props) { - import ConsumerConfig._ - - def this(originalProps: Properties) { - this(new VerifiableProperties(originalProps)) - props.verify() - } - - /** a string that uniquely identifies a set of consumers within the same consumer group */ - val groupId = props.getString("group.id") - - /** consumer id: generated automatically if not set. - * Set this explicitly for only testing purpose. */ - val consumerId: Option[String] = Option(props.getString("consumer.id", null)) - - /** the socket timeout for network requests. Its value should be at least fetch.wait.max.ms. */ - val socketTimeoutMs = props.getInt("socket.timeout.ms", SocketTimeout) - - /** the socket receive buffer for network requests */ - val socketReceiveBufferBytes = props.getInt("socket.receive.buffer.bytes", SocketBufferSize) - - /** the number of bytes of messages to attempt to fetch from each partition */ - val fetchMessageMaxBytes = props.getInt("fetch.message.max.bytes", FetchSize) - - /** the number threads used to fetch data */ - val numConsumerFetchers = props.getInt("num.consumer.fetchers", NumConsumerFetchers) - - /** if true, periodically commit to zookeeper the offset of messages already fetched by the consumer */ - val autoCommitEnable = props.getBoolean("auto.commit.enable", AutoCommit) - - /** the frequency in ms that the consumer offsets are committed to zookeeper */ - val autoCommitIntervalMs = props.getInt("auto.commit.interval.ms", AutoCommitInterval) - - /** max number of message chunks buffered for consumption, each chunk can be up to fetch.message.max.bytes*/ - val queuedMaxMessages = props.getInt("queued.max.message.chunks", MaxQueuedChunks) - - /** max number of retries during rebalance */ - val rebalanceMaxRetries = props.getInt("rebalance.max.retries", MaxRebalanceRetries) - - /** the minimum amount of data the server should return for a fetch request. If insufficient data is available the request will block */ - val fetchMinBytes = props.getInt("fetch.min.bytes", MinFetchBytes) - - /** the maximum amount of data the server should return for a fetch request */ - val fetchMaxBytes = props.getInt("fetch.max.bytes", MaxFetchBytes) - - /** 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) - - /** backoff time to refresh the leader of a partition after it loses the current leader */ - val refreshLeaderBackoffMs = props.getInt("refresh.leader.backoff.ms", RefreshMetadataBackoffMs) - - /** backoff time to reconnect the offsets channel or to retry offset fetches/commits */ - val offsetsChannelBackoffMs = props.getInt("offsets.channel.backoff.ms", OffsetsChannelBackoffMs) - /** socket timeout to use when reading responses for Offset Fetch/Commit requests. This timeout will also be used for - * the ConsumerMetdata requests that are used to query for the offset coordinator. */ - val offsetsChannelSocketTimeoutMs = props.getInt("offsets.channel.socket.timeout.ms", OffsetsChannelSocketTimeoutMs) - - /** Retry the offset commit up to this many times on failure. This retry count only applies to offset commits during - * shut-down. It does not apply to commits from the auto-commit thread. It also does not apply to attempts to query - * for the offset coordinator before committing offsets. i.e., if a consumer metadata request fails for any reason, - * it is retried and that retry does not count toward this limit. */ - val offsetsCommitMaxRetries = props.getInt("offsets.commit.max.retries", OffsetsCommitMaxRetries) - - /** Specify whether offsets should be committed to "zookeeper" (default) or "kafka" */ - val offsetsStorage = props.getString("offsets.storage", OffsetsStorage).toLowerCase(Locale.ROOT) - - /** If you are using "kafka" as offsets.storage, you can dual commit offsets to ZooKeeper (in addition to Kafka). This - * is required during migration from zookeeper-based offset storage to kafka-based offset storage. With respect to any - * given consumer group, it is safe to turn this off after all instances within that group have been migrated to - * the new jar that commits offsets to the broker (instead of directly to ZooKeeper). */ - val dualCommitEnabled = props.getBoolean("dual.commit.enabled", offsetsStorage == "kafka") - - /* what to do if an offset is out of range. - smallest : automatically reset the offset to the smallest offset - largest : automatically reset the offset to the largest offset - anything else: throw exception to the consumer */ - val autoOffsetReset = props.getString("auto.offset.reset", AutoOffsetReset) - - /** throw a timeout exception to the consumer if no message is available for consumption after the specified interval */ - val consumerTimeoutMs = props.getInt("consumer.timeout.ms", ConsumerTimeoutMs) - - /** - * Client id is specified by the kafka consumer client, used to distinguish different clients - */ - val clientId = props.getString("client.id", groupId) - - /** Whether messages from internal topics (such as offsets) should be exposed to the consumer. */ - val excludeInternalTopics = props.getBoolean("exclude.internal.topics", ExcludeInternalTopics) - - /** Select a strategy for assigning partitions to consumer streams. Possible values: range, roundrobin */ - val partitionAssignmentStrategy = props.getString("partition.assignment.strategy", DefaultPartitionAssignmentStrategy) - - validate(this) -} - diff --git a/core/src/main/scala/kafka/consumer/ConsumerConnector.scala b/core/src/main/scala/kafka/consumer/ConsumerConnector.scala deleted file mode 100644 index f6d4a74a4d7bf..0000000000000 --- a/core/src/main/scala/kafka/consumer/ConsumerConnector.scala +++ /dev/null @@ -1,126 +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.consumer - -import kafka.common.{OffsetAndMetadata, TopicAndPartition} -import kafka.javaapi.consumer.ConsumerRebalanceListener - -import scala.collection._ -import kafka.utils.Logging -import kafka.serializer._ - -/** - * Main interface for consumer - */ -@deprecated("This trait has been deprecated and will be removed in a future release.", "0.11.0.0") -trait ConsumerConnector { - - /** - * Create a list of MessageStreams for each topic. - * - * @param topicCountMap a map of (topic, #streams) pair - * @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. - */ - def createMessageStreams(topicCountMap: Map[String,Int]): Map[String, List[KafkaStream[Array[Byte],Array[Byte]]]] - - /** - * Create a list of MessageStreams for each topic. - * - * @param topicCountMap a map of (topic, #streams) pair - * @param keyDecoder Decoder to decode the key portion of the message - * @param valueDecoder Decoder to decode the value portion of the message - * @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. - */ - def createMessageStreams[K,V](topicCountMap: Map[String,Int], - keyDecoder: Decoder[K], - valueDecoder: Decoder[V]) - : Map[String,List[KafkaStream[K,V]]] - - /** - * Create a list of message streams for all topics that match a given filter. - * - * @param topicFilter Either a Whitelist or Blacklist TopicFilter object. - * @param numStreams Number of streams to return - * @param keyDecoder Decoder to decode the key portion of the message - * @param valueDecoder Decoder to decode the value portion of the message - * @return a list of KafkaStream each of which provides an - * iterator over message/metadata pairs over allowed topics. - */ - def createMessageStreamsByFilter[K,V](topicFilter: TopicFilter, - numStreams: Int = 1, - keyDecoder: Decoder[K] = new DefaultDecoder(), - valueDecoder: Decoder[V] = new DefaultDecoder()) - : Seq[KafkaStream[K,V]] - - /** - * Commit the offsets of all broker partitions connected by this connector. - */ - def commitOffsets(retryOnFailure: Boolean) - - /** - * KAFKA-1743: This method added for backward compatibility. - */ - def commitOffsets() - - /** - * Commit offsets from an external offsets map. - * @param offsetsToCommit the offsets to be committed. - */ - def commitOffsets(offsetsToCommit: immutable.Map[TopicAndPartition, OffsetAndMetadata], retryOnFailure: Boolean) - - /** - * Wire in a consumer rebalance listener to be executed when consumer rebalance occurs. - * @param listener The consumer rebalance listener to wire in - */ - def setConsumerRebalanceListener(listener: ConsumerRebalanceListener) - - /** - * Shut down the connector - */ - def shutdown() -} - -@deprecated("This object has been deprecated and will be removed in a future release. " + - "Please use org.apache.kafka.clients.consumer.Consumer instead.", "0.11.0.0") -object Consumer extends Logging { - /** - * Create a ConsumerConnector - * - * @param config at the minimum, need to specify the groupid of the consumer and the zookeeper - * connection string zookeeper.connect. - */ - def create(config: ConsumerConfig): ConsumerConnector = { - val consumerConnect = new ZookeeperConsumerConnector(config) - consumerConnect - } - - /** - * Create a ConsumerConnector - * - * @param config at the minimum, need to specify the groupid of the consumer and the zookeeper - * connection string zookeeper.connect. - */ - def createJavaConsumerConnector(config: ConsumerConfig): kafka.javaapi.consumer.ConsumerConnector = { - val consumerConnect = new kafka.javaapi.consumer.ZookeeperConsumerConnector(config) - consumerConnect - } -} diff --git a/core/src/main/scala/kafka/consumer/ConsumerFetcherManager.scala b/core/src/main/scala/kafka/consumer/ConsumerFetcherManager.scala deleted file mode 100755 index e84472f06bb96..0000000000000 --- a/core/src/main/scala/kafka/consumer/ConsumerFetcherManager.scala +++ /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 kafka.consumer - -import kafka.server.{AbstractFetcherManager, AbstractFetcherThread, BrokerAndInitialOffset} -import kafka.cluster.{BrokerEndPoint, Cluster} -import org.apache.kafka.common.TopicPartition -import org.apache.kafka.common.utils.Time - -import scala.collection.immutable -import collection.mutable.HashMap -import scala.collection.mutable -import java.util.concurrent.locks.ReentrantLock - -import kafka.utils.CoreUtils.inLock -import kafka.utils.ZkUtils -import kafka.utils.ShutdownableThread -import kafka.client.ClientUtils -import java.util.concurrent.atomic.AtomicInteger - -/** - * Usage: - * Once ConsumerFetcherManager is created, startConnections() and stopAllConnections() can be called repeatedly - * until shutdown() is called. - */ -@deprecated("This class has been deprecated and will be removed in a future release.", "0.11.0.0") -class ConsumerFetcherManager(private val consumerIdString: String, - private val config: ConsumerConfig, - private val zkUtils : ZkUtils) - extends AbstractFetcherManager("ConsumerFetcherManager-%d".format(Time.SYSTEM.milliseconds), - config.clientId, config.numConsumerFetchers) { - private var partitionMap: immutable.Map[TopicPartition, PartitionTopicInfo] = null - private val noLeaderPartitionSet = new mutable.HashSet[TopicPartition] - private val lock = new ReentrantLock - private val cond = lock.newCondition() - private var leaderFinderThread: ShutdownableThread = null - private val correlationId = new AtomicInteger(0) - - 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[TopicPartition, BrokerEndPoint] - lock.lock() - try { - while (noLeaderPartitionSet.isEmpty) { - trace("No partition for leader election.") - cond.await() - } - - trace("Partitions without leader %s".format(noLeaderPartitionSet)) - val brokers = ClientUtils.getPlaintextBrokerEndPoints(zkUtils) - val topicsMetadata = ClientUtils.fetchTopicMetadata(noLeaderPartitionSet.map(m => m.topic).toSet, - brokers, - config.clientId, - config.socketTimeoutMs, - correlationId.getAndIncrement).topicsMetadata - if(isDebugEnabled) topicsMetadata.foreach(topicMetadata => debug(topicMetadata.toString())) - topicsMetadata.foreach { tmd => - val topic = tmd.topic - tmd.partitionsMetadata.foreach { pmd => - val topicAndPartition = new TopicPartition(topic, pmd.partitionId) - if(pmd.leader.isDefined && noLeaderPartitionSet.contains(topicAndPartition)) { - val leaderBroker = pmd.leader.get - leaderForPartitionsMap.put(topicAndPartition, leaderBroker) - noLeaderPartitionSet -= topicAndPartition - } - } - } - } catch { - case t: Throwable => { - if (!isRunning) - throw t /* If this thread is stopped, propagate this exception to kill the thread. */ - else - warn("Failed to find leader for %s".format(noLeaderPartitionSet), t) - } - } finally { - lock.unlock() - } - - try { - addFetcherForPartitions(leaderForPartitionsMap.map { case (topicPartition, broker) => - topicPartition -> BrokerAndInitialOffset(broker, partitionMap(topicPartition).getFetchOffset())} - ) - } catch { - case t: Throwable => - if (!isRunning) - throw t /* If this thread is stopped, propagate this exception to kill the thread. */ - else { - warn("Failed to add leader for partitions %s; will retry".format(leaderForPartitionsMap.keySet.mkString(",")), t) - lock.lock() - noLeaderPartitionSet ++= leaderForPartitionsMap.keySet - lock.unlock() - } - } - - shutdownIdleFetcherThreads() - Thread.sleep(config.refreshLeaderBackoffMs) - } - } - - override def createFetcherThread(fetcherId: Int, sourceBroker: BrokerEndPoint): AbstractFetcherThread = { - new ConsumerFetcherThread(consumerIdString, fetcherId, config, sourceBroker, partitionMap, this) - } - - def startConnections(topicInfos: Iterable[PartitionTopicInfo], cluster: Cluster) { - leaderFinderThread = new LeaderFinderThread(consumerIdString + "-leader-finder-thread") - leaderFinderThread.start() - - inLock(lock) { - partitionMap = topicInfos.map(tpi => (new TopicPartition(tpi.topic, tpi.partitionId), tpi)).toMap - noLeaderPartitionSet ++= topicInfos.map(tpi => new TopicPartition(tpi.topic, tpi.partitionId)) - cond.signalAll() - } - } - - def stopConnections() { - /* - * Stop the leader finder thread first before stopping fetchers. Otherwise, if there are more partitions without - * leader, then the leader finder thread will process these partitions (before shutting down) and add fetchers for - * these partitions. - */ - info("Stopping leader finder thread") - if (leaderFinderThread != null) { - leaderFinderThread.shutdown() - leaderFinderThread = null - } - - info("Stopping all fetchers") - closeAllFetchers() - - // no need to hold the lock for the following since leaderFindThread and all fetchers have been stopped - partitionMap = null - noLeaderPartitionSet.clear() - - info("All connections stopped") - } - - def addPartitionsWithError(partitionList: Iterable[TopicPartition]) { - debug("adding partitions with error %s".format(partitionList)) - inLock(lock) { - if (partitionMap != null) { - noLeaderPartitionSet ++= partitionList - cond.signalAll() - } - } - } -} diff --git a/core/src/main/scala/kafka/consumer/ConsumerFetcherThread.scala b/core/src/main/scala/kafka/consumer/ConsumerFetcherThread.scala deleted file mode 100644 index 9426884629670..0000000000000 --- a/core/src/main/scala/kafka/consumer/ConsumerFetcherThread.scala +++ /dev/null @@ -1,158 +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.consumer - -import kafka.api.{FetchRequestBuilder, FetchResponsePartitionData, OffsetRequest, Request} -import kafka.cluster.BrokerEndPoint -import kafka.message.ByteBufferMessageSet -import kafka.server.{AbstractFetcherThread, PartitionFetchState, OffsetTruncationState} -import AbstractFetcherThread.ResultWithPartitions -import kafka.common.{ErrorMapping, TopicAndPartition} - -import scala.collection.Map -import ConsumerFetcherThread._ -import org.apache.kafka.common.TopicPartition -import org.apache.kafka.common.protocol.Errors -import org.apache.kafka.common.record.MemoryRecords -import org.apache.kafka.common.requests.EpochEndOffset - - -@deprecated("This class has been deprecated and will be removed in a future release. " + - "Please use org.apache.kafka.clients.consumer.internals.Fetcher instead.", "0.11.0.0") -class ConsumerFetcherThread(consumerIdString: String, - fetcherId: Int, - val config: ConsumerConfig, - sourceBroker: BrokerEndPoint, - partitionMap: Map[TopicPartition, PartitionTopicInfo], - val consumerFetcherManager: ConsumerFetcherManager) - extends AbstractFetcherThread(name = s"ConsumerFetcherThread-$consumerIdString-$fetcherId-${sourceBroker.id}", - clientId = config.clientId, - sourceBroker = sourceBroker, - fetchBackOffMs = config.refreshLeaderBackoffMs, - isInterruptible = true, - includeLogTruncation = false) { - - type REQ = FetchRequest - type PD = PartitionData - - this.logIdent = s"[ConsumerFetcher consumerId=$consumerIdString, leaderId=${sourceBroker.id}, " + - s"fetcherId=$fetcherId] " - - 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(3) // for now, the old consumer is pinned to the old message format through the fetch request - - 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(topicPartition: TopicPartition, fetchOffset: Long, partitionData: PartitionData) { - val pti = partitionMap(topicPartition) - if (pti.getFetchOffset != fetchOffset) - throw new RuntimeException("Offset doesn't match for partition [%s,%d] pti offset: %d fetch offset: %d" - .format(topicPartition.topic, topicPartition.partition, pti.getFetchOffset, fetchOffset)) - pti.enqueue(partitionData.underlying.messages.asInstanceOf[ByteBufferMessageSet]) - } - - // handle a partition whose offset is out of range and return a new fetch offset - def handleOffsetOutOfRange(topicPartition: TopicPartition): Long = { - val startTimestamp = config.autoOffsetReset match { - case OffsetRequest.SmallestTimeString => OffsetRequest.EarliestTime - case _ => OffsetRequest.LatestTime - } - val topicAndPartition = TopicAndPartition(topicPartition.topic, topicPartition.partition) - val newOffset = simpleConsumer.earliestOrLatestOffset(topicAndPartition, startTimestamp, Request.OrdinaryConsumerId) - val pti = partitionMap(topicPartition) - pti.resetFetchOffset(newOffset) - pti.resetConsumeOffset(newOffset) - newOffset - } - - // any logic for partitions whose leader has changed - def handlePartitionsWithErrors(partitions: Iterable[TopicPartition]) { - if (partitions.nonEmpty) { - removePartitions(partitions.toSet) - consumerFetcherManager.addPartitionsWithError(partitions) - } - } - - protected def buildFetchRequest(partitionMap: collection.Seq[(TopicPartition, PartitionFetchState)]): ResultWithPartitions[FetchRequest] = { - partitionMap.foreach { case ((topicPartition, partitionFetchState)) => - if (partitionFetchState.isReadyForFetch) - fetchRequestBuilder.addFetch(topicPartition.topic, topicPartition.partition, partitionFetchState.fetchOffset, fetchSize) - } - - ResultWithPartitions(new FetchRequest(fetchRequestBuilder.build()), Set()) - } - - protected def fetch(fetchRequest: FetchRequest): Seq[(TopicPartition, PartitionData)] = - simpleConsumer.fetch(fetchRequest.underlying).data.map { case (TopicAndPartition(t, p), value) => - new TopicPartition(t, p) -> new PartitionData(value) - } - - override def buildLeaderEpochRequest(allPartitions: Seq[(TopicPartition, PartitionFetchState)]): ResultWithPartitions[Map[TopicPartition, Int]] = { - ResultWithPartitions(Map(), Set()) - } - - override def fetchEpochsFromLeader(partitions: Map[TopicPartition, Int]): Map[TopicPartition, EpochEndOffset] = { Map() } - - override def maybeTruncate(fetchedEpochs: Map[TopicPartition, EpochEndOffset]): ResultWithPartitions[Map[TopicPartition, OffsetTruncationState]] = { - ResultWithPartitions(Map(), Set()) - } -} - -@deprecated("This object has been deprecated and will be removed in a future release. " + - "Please use org.apache.kafka.clients.consumer.internals.Fetcher instead.", "0.11.0.0") -object ConsumerFetcherThread { - - class FetchRequest(val underlying: kafka.api.FetchRequest) extends AbstractFetcherThread.FetchRequest { - private lazy val tpToOffset: Map[TopicPartition, Long] = underlying.requestInfo.map { case (tp, fetchInfo) => - new TopicPartition(tp.topic, tp.partition) -> fetchInfo.offset - }.toMap - def isEmpty: Boolean = underlying.requestInfo.isEmpty - def offset(topicPartition: TopicPartition): Long = tpToOffset(topicPartition) - override def toString = underlying.toString - } - - class PartitionData(val underlying: FetchResponsePartitionData) extends AbstractFetcherThread.PartitionData { - def error = underlying.error - def toRecords: MemoryRecords = underlying.messages.asInstanceOf[ByteBufferMessageSet].asRecords - def highWatermark: Long = underlying.hw - def exception: Option[Throwable] = - if (error == Errors.NONE) None else Some(ErrorMapping.exceptionFor(error.code)) - override def toString = underlying.toString - } -} diff --git a/core/src/main/scala/kafka/consumer/ConsumerIterator.scala b/core/src/main/scala/kafka/consumer/ConsumerIterator.scala deleted file mode 100755 index f096c55af3102..0000000000000 --- a/core/src/main/scala/kafka/consumer/ConsumerIterator.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.consumer - -import kafka.utils.{IteratorTemplate, Logging} -import java.util.concurrent.{TimeUnit, BlockingQueue} -import kafka.serializer.Decoder -import java.util.concurrent.atomic.AtomicReference -import kafka.message.{MessageAndOffset, MessageAndMetadata} -import kafka.common.{KafkaException, MessageSizeTooLargeException} - - -/** - * An iterator that blocks until a value can be read from the supplied queue. - * The iterator takes a shutdownCommand object which can be added to the queue to trigger a shutdown - * - */ -@deprecated("This class has been deprecated and will be removed in a future release.", "0.11.0.0") -class ConsumerIterator[K, V](private val channel: BlockingQueue[FetchedDataChunk], - consumerTimeoutMs: Int, - private val keyDecoder: Decoder[K], - private val valueDecoder: Decoder[V], - val clientId: String) - extends IteratorTemplate[MessageAndMetadata[K, V]] with Logging { - - 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) - - override def next(): MessageAndMetadata[K, V] = { - val item = super.next() - if(consumedOffset < 0) - throw new KafkaException("Offset returned by the message set is invalid %d".format(consumedOffset)) - currentTopicInfo.resetConsumeOffset(consumedOffset) - val topic = currentTopicInfo.topic - trace("Setting %s consumed offset to %d".format(topic, consumedOffset)) - consumerTopicStats.getConsumerTopicStats(topic).messageRate.mark() - consumerTopicStats.getConsumerAllTopicStats().messageRate.mark() - item - } - - protected def makeNext(): MessageAndMetadata[K, V] = { - var currentDataChunk: FetchedDataChunk = null - // if we don't have an iterator, get one - var localCurrent = current.get() - if(localCurrent == null || !localCurrent.hasNext) { - if (consumerTimeoutMs < 0) - currentDataChunk = channel.take - else { - currentDataChunk = channel.poll(consumerTimeoutMs, TimeUnit.MILLISECONDS) - if (currentDataChunk == null) { - // reset state to make the iterator re-iterable - resetState() - throw new ConsumerTimeoutException - } - } - if(currentDataChunk eq ZookeeperConsumerConnector.shutdownCommand) { - debug("Received the shutdown command") - return allDone - } else { - currentTopicInfo = currentDataChunk.topicInfo - val cdcFetchOffset = currentDataChunk.fetchOffset - val ctiConsumeOffset = currentTopicInfo.getConsumeOffset - if (ctiConsumeOffset < cdcFetchOffset) { - error("consumed offset: %d doesn't match fetch offset: %d for %s;\n Consumer may lose data" - .format(ctiConsumeOffset, cdcFetchOffset, currentTopicInfo)) - currentTopicInfo.resetConsumeOffset(cdcFetchOffset) - } - localCurrent = currentDataChunk.messages.iterator - - current.set(localCurrent) - } - // if we just updated the current chunk and it is empty that means the fetch size is too small! - if(currentDataChunk.messages.validBytes == 0) - throw new MessageSizeTooLargeException("Found a message larger than the maximum fetch size of this consumer on topic " + - "%s partition %d at fetch offset %d. Increase the fetch size, or decrease the maximum message size the broker will allow." - .format(currentDataChunk.topicInfo.topic, currentDataChunk.topicInfo.partitionId, currentDataChunk.fetchOffset)) - } - var item = localCurrent.next() - // reject the messages that have already been consumed - while (item.offset < currentTopicInfo.getConsumeOffset && localCurrent.hasNext) { - item = localCurrent.next() - } - consumedOffset = item.nextOffset - - item.message.ensureValid() // validate checksum of message to ensure it is valid - - new MessageAndMetadata(currentTopicInfo.topic, - currentTopicInfo.partitionId, - item.message, - item.offset, - keyDecoder, - valueDecoder, - item.message.timestamp, - item.message.timestampType) - } - - def clearCurrentChunk() { - debug("Clearing the current data chunk for this consumer iterator") - current.set(null) - } -} - -@deprecated("This class has been deprecated and will be removed in a future release. " + - "Please use org.apache.kafka.common.errors.TimeoutException instead.", "0.11.0.0") -class ConsumerTimeoutException() extends RuntimeException() - diff --git a/core/src/main/scala/kafka/consumer/ConsumerTopicStats.scala b/core/src/main/scala/kafka/consumer/ConsumerTopicStats.scala deleted file mode 100644 index d13b327e9c15f..0000000000000 --- a/core/src/main/scala/kafka/consumer/ConsumerTopicStats.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.consumer - -import kafka.utils.{Pool, threadsafe, Logging} -import java.util.concurrent.TimeUnit -import kafka.metrics.KafkaMetricsGroup -import kafka.common.{ClientIdTopic, ClientIdAllTopics, ClientIdAndTopic} - -@threadsafe -@deprecated("This class has been deprecated and will be removed in a future release.", "0.11.0.0") -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) -} - -/** - * Tracks metrics for each topic the given consumer client has consumed data from. - * @param clientId The clientId of the given consumer client. - */ -@deprecated("This class has been deprecated and will be removed in a future release.", "0.11.0.0") -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 ClientIdAllTopics(clientId)) // to differentiate from a topic named AllTopics - - def getConsumerAllTopicStats(): ConsumerTopicMetrics = allTopicStats - - def getConsumerTopicStats(topic: String): ConsumerTopicMetrics = { - stats.getAndMaybePut(new ClientIdAndTopic(clientId, topic)) - } -} - -/** - * Stores the topic stats information of each consumer client in a (clientId -> ConsumerTopicStats) map. - */ -@deprecated("This object has been deprecated and will be removed in a future release.", "0.11.0.0") -object ConsumerTopicStatsRegistry { - private val valueFactory = (k: String) => new ConsumerTopicStats(k) - private val globalStats = new Pool[String, ConsumerTopicStats](Some(valueFactory)) - - def getConsumerTopicStat(clientId: String) = { - globalStats.getAndMaybePut(clientId) - } - - def removeConsumerTopicStat(clientId: String) { - globalStats.remove(clientId) - } -} \ No newline at end of file diff --git a/core/src/main/scala/kafka/consumer/FetchRequestAndResponseStats.scala b/core/src/main/scala/kafka/consumer/FetchRequestAndResponseStats.scala deleted file mode 100644 index 462a85ba063e6..0000000000000 --- a/core/src/main/scala/kafka/consumer/FetchRequestAndResponseStats.scala +++ /dev/null @@ -1,82 +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.consumer - -import java.util.concurrent.TimeUnit - -import kafka.common.{ClientIdAllBrokers, ClientIdBroker, ClientIdAndBroker} -import kafka.metrics.{KafkaMetricsGroup, KafkaTimer} -import kafka.utils.Pool - -@deprecated("This class has been deprecated and will be removed in a future release.", "0.11.0.0") -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) -} - -/** - * Tracks metrics of the requests made by a given consumer client to all brokers, and the responses obtained from the brokers. - * @param clientId ClientId of the given consumer - */ -@deprecated("This class has been deprecated and will be removed in a future release.", "0.11.0.0") -class FetchRequestAndResponseStats(clientId: String) { - 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(brokerHost: String, brokerPort: Int): FetchRequestAndResponseMetrics = { - stats.getAndMaybePut(new ClientIdAndBroker(clientId, brokerHost, brokerPort)) - } -} - -/** - * Stores the fetch request and response stats information of each consumer client in a (clientId -> FetchRequestAndResponseStats) map. - */ -@deprecated("This object has been deprecated and will be removed in a future release.", "0.11.0.0") -object FetchRequestAndResponseStatsRegistry { - private val valueFactory = (k: String) => new FetchRequestAndResponseStats(k) - private val globalStats = new Pool[String, FetchRequestAndResponseStats](Some(valueFactory)) - - def getFetchRequestAndResponseStats(clientId: String) = { - globalStats.getAndMaybePut(clientId) - } - - def removeConsumerFetchRequestAndResponseStats(clientId: String) { - val pattern = (".*" + clientId + ".*").r - val keys = globalStats.keys - for (key <- keys) { - pattern.findFirstIn(key) match { - case Some(_) => globalStats.remove(key) - case _ => - } - } - } -} - - diff --git a/core/src/main/scala/kafka/consumer/KafkaStream.scala b/core/src/main/scala/kafka/consumer/KafkaStream.scala deleted file mode 100644 index 914cedd07fd54..0000000000000 --- a/core/src/main/scala/kafka/consumer/KafkaStream.scala +++ /dev/null @@ -1,53 +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.consumer - - -import java.util.concurrent.BlockingQueue -import kafka.serializer.Decoder -import kafka.message.MessageAndMetadata - -@deprecated("This class has been deprecated and will be removed in a future release. " + - "Please use org.apache.kafka.streams.KafkaStreams instead.", "0.11.0.0") -class KafkaStream[K,V](private val queue: BlockingQueue[FetchedDataChunk], - consumerTimeoutMs: Int, - private val keyDecoder: Decoder[K], - private val valueDecoder: Decoder[V], - val clientId: String) - extends Iterable[MessageAndMetadata[K,V]] with java.lang.Iterable[MessageAndMetadata[K,V]] { - - private val iter: ConsumerIterator[K,V] = - new ConsumerIterator[K,V](queue, consumerTimeoutMs, keyDecoder, valueDecoder, clientId) - - /** - * Create an iterator over messages in the stream. - */ - def iterator: ConsumerIterator[K,V] = iter - - /** - * This method clears the queue being iterated during the consumer rebalancing. This is mainly - * to reduce the number of duplicates received by the consumer - */ - def clear() { - iter.clearCurrentChunk() - } - - override def toString: String = { - "%s kafka stream".format(clientId) - } -} diff --git a/core/src/main/scala/kafka/consumer/PartitionAssignor.scala b/core/src/main/scala/kafka/consumer/PartitionAssignor.scala deleted file mode 100755 index 7d49b994a96d5..0000000000000 --- a/core/src/main/scala/kafka/consumer/PartitionAssignor.scala +++ /dev/null @@ -1,172 +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.consumer - -import kafka.common.TopicAndPartition -import kafka.utils.{Pool, CoreUtils, ZkUtils, Logging} - -import scala.collection.mutable - -@deprecated("This trait has been deprecated and will be removed in a future release. " + - "Please use org.apache.kafka.clients.consumer.internals.PartitionAssignor instead.", "0.11.0.0") -trait PartitionAssignor { - - /** - * Assigns partitions to consumer instances in a group. - * @return An assignment map of partition to this consumer group. This includes assignments for threads that belong - * to the same consumer group. - */ - def assign(ctx: AssignmentContext): Pool[String, mutable.Map[TopicAndPartition, ConsumerThreadId]] - -} - -@deprecated("This object has been deprecated and will be removed in a future release. " + - "Please use org.apache.kafka.clients.consumer.internals.PartitionAssignor instead.", "0.11.0.0") -object PartitionAssignor { - def createInstance(assignmentStrategy: String) = assignmentStrategy match { - case "roundrobin" => new RoundRobinAssignor() - case _ => new RangeAssignor() - } -} - -@deprecated("This class has been deprecated and will be removed in a future release.", "0.11.0.0") -class AssignmentContext(group: String, val consumerId: String, excludeInternalTopics: Boolean, zkUtils: ZkUtils) { - val myTopicThreadIds: collection.Map[String, collection.Set[ConsumerThreadId]] = { - val myTopicCount = TopicCount.constructTopicCount(group, consumerId, zkUtils, excludeInternalTopics) - myTopicCount.getConsumerThreadIdsPerTopic - } - - val consumersForTopic: collection.Map[String, List[ConsumerThreadId]] = - zkUtils.getConsumersPerTopic(group, excludeInternalTopics) - - // Some assignment strategies require knowledge of all topics consumed by any member of the group - val partitionsForTopic: collection.Map[String, Seq[Int]] = - zkUtils.getPartitionsForTopics(consumersForTopic.keySet.toSeq) - - val consumers: Seq[String] = zkUtils.getConsumersInGroup(group).sorted -} - -/** - * The round-robin partition assignor lays out all the available partitions and all the available consumer threads. It - * then proceeds to do a round-robin assignment from partition to consumer thread. If the subscriptions of all consumer - * instances are identical, then the partitions will be uniformly distributed. (i.e., the partition ownership counts - * will be within a delta of exactly one across all consumer threads.) - */ -@deprecated("This class has been deprecated and will be removed in a future release. " + - "Please use org.apache.kafka.clients.consumer.RoundRobinAssignor instead.", "0.11.0.0") -class RoundRobinAssignor() extends PartitionAssignor with Logging { - - def assign(ctx: AssignmentContext) = { - - val valueFactory = (_: String) => new mutable.HashMap[TopicAndPartition, ConsumerThreadId] - val partitionAssignment = - new Pool[String, mutable.Map[TopicAndPartition, ConsumerThreadId]](Some(valueFactory)) - - if (ctx.consumersForTopic.nonEmpty) { - // Collect consumer thread ids across all topics, remove duplicates, and sort to ensure determinism - val allThreadIds = ctx.consumersForTopic.flatMap { case (_, threadIds) => - threadIds - }.toSet.toSeq.sorted - - val threadAssignor = CoreUtils.circularIterator(allThreadIds) - - info("Starting round-robin assignment with consumers " + ctx.consumers) - val allTopicPartitions = ctx.partitionsForTopic.flatMap { case (topic, partitions) => - info("Consumer %s rebalancing the following partitions for topic %s: %s" - .format(ctx.consumerId, topic, partitions)) - partitions.map(partition => { - TopicAndPartition(topic, partition) - }) - }.toSeq.sortWith((topicPartition1, topicPartition2) => { - /* - * Randomize the order by taking the hashcode to reduce the likelihood of all partitions of a given topic ending - * up on one consumer (if it has a high enough stream count). - */ - topicPartition1.toString.hashCode < topicPartition2.toString.hashCode - }) - - allTopicPartitions.foreach(topicPartition => { - val threadId = threadAssignor.dropWhile(threadId => !ctx.consumersForTopic(topicPartition.topic).contains(threadId)).next - // record the partition ownership decision - val assignmentForConsumer = partitionAssignment.getAndMaybePut(threadId.consumer) - assignmentForConsumer += (topicPartition -> threadId) - }) - } - - // assign Map.empty for the consumers which are not associated with topic partitions - ctx.consumers.foreach(consumerId => partitionAssignment.getAndMaybePut(consumerId)) - partitionAssignment - } -} - -/** - * Range partitioning works on a per-topic basis. For each topic, we lay out the available partitions in numeric order - * and the consumer threads in lexicographic order. We then divide the number of partitions by the total number of - * consumer streams (threads) to determine the number of partitions to assign to each consumer. If it does not evenly - * divide, then the first few consumers will have one extra partition. For example, suppose there are two consumers C1 - * and C2 with two streams each, and there are five available partitions (p0, p1, p2, p3, p4). So each consumer thread - * will get at least one partition and the first consumer thread will get one extra partition. So the assignment will be: - * p0 -> C1-0, p1 -> C1-0, p2 -> C1-1, p3 -> C2-0, p4 -> C2-1 - */ -@deprecated("This class has been deprecated and will be removed in a future release. " + - "Please use org.apache.kafka.clients.consumer.RangeAssignor instead.", "0.11.0.0") -class RangeAssignor() extends PartitionAssignor with Logging { - - def assign(ctx: AssignmentContext) = { - val valueFactory = (_: String) => new mutable.HashMap[TopicAndPartition, ConsumerThreadId] - val partitionAssignment = - new Pool[String, mutable.Map[TopicAndPartition, ConsumerThreadId]](Some(valueFactory)) - for (topic <- ctx.myTopicThreadIds.keySet) { - val curConsumers = ctx.consumersForTopic(topic) - val curPartitions: Seq[Int] = ctx.partitionsForTopic(topic) - - val nPartsPerConsumer = curPartitions.size / curConsumers.size - val nConsumersWithExtraPart = curPartitions.size % curConsumers.size - - info("Consumer " + ctx.consumerId + " rebalancing the following partitions: " + curPartitions + - " for topic " + topic + " with consumers: " + curConsumers) - - for (consumerThreadId <- curConsumers) { - val myConsumerPosition = curConsumers.indexOf(consumerThreadId) - assert(myConsumerPosition >= 0) - val startPart = nPartsPerConsumer * myConsumerPosition + myConsumerPosition.min(nConsumersWithExtraPart) - val nParts = nPartsPerConsumer + (if (myConsumerPosition + 1 > nConsumersWithExtraPart) 0 else 1) - - /** - * Range-partition the sorted partitions to consumers for better locality. - * The first few consumers pick up an extra partition, if any. - */ - if (nParts <= 0) - warn("No broker partitions consumed by consumer thread " + consumerThreadId + " for topic " + topic) - else { - for (i <- startPart until startPart + nParts) { - val partition = curPartitions(i) - info(consumerThreadId + " attempting to claim partition " + partition) - // record the partition ownership decision - val assignmentForConsumer = partitionAssignment.getAndMaybePut(consumerThreadId.consumer) - assignmentForConsumer += (TopicAndPartition(topic, partition) -> consumerThreadId) - } - } - } - } - - // assign Map.empty for the consumers which are not associated with topic partitions - ctx.consumers.foreach(consumerId => partitionAssignment.getAndMaybePut(consumerId)) - partitionAssignment - } -} diff --git a/core/src/main/scala/kafka/consumer/PartitionTopicInfo.scala b/core/src/main/scala/kafka/consumer/PartitionTopicInfo.scala deleted file mode 100644 index 9a0879a198730..0000000000000 --- a/core/src/main/scala/kafka/consumer/PartitionTopicInfo.scala +++ /dev/null @@ -1,80 +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.consumer - -import java.util.concurrent._ -import java.util.concurrent.atomic._ -import kafka.message._ -import kafka.utils.Logging - -@deprecated("This class has been deprecated and will be removed in a future release.", "0.11.0.0") -class PartitionTopicInfo(val topic: String, - val partitionId: Int, - private val chunkQueue: BlockingQueue[FetchedDataChunk], - private val consumedOffset: AtomicLong, - private val fetchedOffset: AtomicLong, - private val fetchSize: AtomicInteger, - private val clientId: String) extends Logging { - - debug("initial consumer offset of " + this + " is " + consumedOffset.get) - debug("initial fetch offset of " + this + " is " + fetchedOffset.get) - - private val consumerTopicStats = ConsumerTopicStatsRegistry.getConsumerTopicStat(clientId) - - def getConsumeOffset() = consumedOffset.get - - def getFetchOffset() = fetchedOffset.get - - def resetConsumeOffset(newConsumeOffset: Long) = { - consumedOffset.set(newConsumeOffset) - debug("reset consume offset of " + this + " to " + newConsumeOffset) - } - - def resetFetchOffset(newFetchOffset: Long) = { - fetchedOffset.set(newFetchOffset) - debug("reset fetch offset of ( %s ) to %d".format(this, newFetchOffset)) - } - - /** - * Enqueue a message set for processing. - */ - def enqueue(messages: ByteBufferMessageSet) { - val size = messages.validBytes - if(size > 0) { - val next = messages.shallowIterator.toSeq.last.nextOffset - trace("Updating fetch offset = " + fetchedOffset.get + " to " + next) - chunkQueue.put(new FetchedDataChunk(messages, this, fetchedOffset.get)) - fetchedOffset.set(next) - debug("updated fetch offset of (%s) to %d".format(this, next)) - consumerTopicStats.getConsumerTopicStats(topic).byteRate.mark(size) - consumerTopicStats.getConsumerAllTopicStats().byteRate.mark(size) - } else if(messages.sizeInBytes > 0) { - chunkQueue.put(new FetchedDataChunk(messages, this, fetchedOffset.get)) - } - } - - override def toString: String = topic + ":" + partitionId.toString + ": fetched offset = " + fetchedOffset.get + - ": consumed offset = " + consumedOffset.get -} - -@deprecated("This object has been deprecated and will be removed in a future release.", "0.11.0.0") -object PartitionTopicInfo { - val InvalidOffset = -1L - - def isOffsetInvalid(offset: Long) = offset < 0L -} \ No newline at end of file diff --git a/core/src/main/scala/kafka/consumer/SimpleConsumer.scala b/core/src/main/scala/kafka/consumer/SimpleConsumer.scala deleted file mode 100644 index b30c9cefd96b0..0000000000000 --- a/core/src/main/scala/kafka/consumer/SimpleConsumer.scala +++ /dev/null @@ -1,199 +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.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} -import org.apache.kafka.common.protocol.Errors -import org.apache.kafka.common.utils.Utils._ - -/** - * A consumer of kafka messages - */ -@deprecated("This class has been deprecated and will be removed in a future release. " + - "Please use org.apache.kafka.clients.consumer.KafkaConsumer instead.", "0.11.0.0") -@threadsafe -class SimpleConsumer(val host: String, - val port: Int, - val soTimeout: Int, - val bufferSize: Int, - val clientId: String) extends Logging { - - ConsumerConfig.validateClientId(clientId) - private val lock = new Object() - private val blockingChannel = new BlockingChannel(host, port, bufferSize, BlockingChannel.UseDefaultBufferSize, soTimeout) - private val fetchRequestAndResponseStats = FetchRequestAndResponseStatsRegistry.getFetchRequestAndResponseStats(clientId) - private var isClosed = false - - private def connect(): BlockingChannel = { - close - blockingChannel.connect() - blockingChannel - } - - private def disconnect() = { - debug("Disconnecting from " + formatAddress(host, port)) - blockingChannel.disconnect() - } - - private def reconnect() { - disconnect() - 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() - isClosed = true - } - } - - private def sendRequest(request: RequestOrResponse): NetworkReceive = { - lock synchronized { - 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 error:", e) - // retry once - try { - reconnect() - blockingChannel.send(request) - response = blockingChannel.receive() - } catch { - case e: Throwable => - disconnect() - throw e - } - } - response - } - } - - def send(request: TopicMetadataRequest): TopicMetadataResponse = { - val response = sendRequest(request) - TopicMetadataResponse.readFrom(response.payload()) - } - - def send(request: GroupCoordinatorRequest): GroupCoordinatorResponse = { - val response = sendRequest(request) - GroupCoordinatorResponse.readFrom(response.payload()) - } - - /** - * 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 - */ - def fetch(request: FetchRequest): FetchResponse = { - 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.payload(), request.versionId) - val fetchedSize = fetchResponse.sizeInBytes - 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 - } - - /** - * Get a list of valid offsets (up to maxSize) before the given time. - * @param request a [[kafka.api.OffsetRequest]] object. - * @return a [[kafka.api.OffsetResponse]] object. - */ - 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).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).payload(), request.versionId) - - private def getOrMakeConnection() { - if(!isClosed && !blockingChannel.isConnected) { - connect() - } - } - - /** - * Get the earliest or latest offset of a given topic, partition. - * @param topicAndPartition Topic and partition of which the offset is needed. - * @param earliestOrLatest A value to indicate earliest or latest offset. - * @param consumerId Id of the consumer which could be a consumer client, SimpleConsumerShell or a follower broker. - * @return Requested offset. - */ - def earliestOrLatestOffset(topicAndPartition: TopicAndPartition, earliestOrLatest: Long, consumerId: Int): Long = { - val request = OffsetRequest(requestInfo = Map(topicAndPartition -> PartitionOffsetRequestInfo(earliestOrLatest, 1)), - clientId = clientId, - replicaId = consumerId) - val partitionErrorAndOffset = getOffsetsBefore(request).partitionErrorAndOffsets(topicAndPartition) - val offset = partitionErrorAndOffset.error match { - case Errors.NONE => partitionErrorAndOffset.offsets.head - case _ => throw ErrorMapping.exceptionFor(partitionErrorAndOffset.error.code) - } - offset - } -} - diff --git a/core/src/main/scala/kafka/consumer/TopicCount.scala b/core/src/main/scala/kafka/consumer/TopicCount.scala deleted file mode 100755 index 2cabcaea0b746..0000000000000 --- a/core/src/main/scala/kafka/consumer/TopicCount.scala +++ /dev/null @@ -1,145 +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.consumer - -import scala.collection._ -import kafka.utils.{Json, ZKGroupDirs, ZkUtils, Logging, CoreUtils} -import kafka.common.KafkaException - -@deprecated("This trait has been deprecated and will be removed in a future release.", "0.11.0.0") -private[kafka] trait TopicCount { - - def getConsumerThreadIdsPerTopic: Map[String, Set[ConsumerThreadId]] - def getTopicCountMap: Map[String, Int] - def pattern: String - -} - -@deprecated("This class has been deprecated and will be removed in a future release.", "0.11.0.0") -case class ConsumerThreadId(consumer: String, threadId: Int) extends Ordered[ConsumerThreadId] { - override def toString = "%s-%d".format(consumer, threadId) - - def compare(that: ConsumerThreadId) = toString.compare(that.toString) -} - -@deprecated("This object has been deprecated and will be removed in a future release.", "0.11.0.0") -private[kafka] object TopicCount extends Logging { - val whiteListPattern = "white_list" - val blackListPattern = "black_list" - val staticPattern = "static" - - def makeThreadId(consumerIdString: String, threadId: Int) = consumerIdString + "-" + threadId - - def makeConsumerThreadIdsPerTopic(consumerIdString: String, - topicCountMap: Map[String, Int]) = { - val consumerThreadIdsPerTopicMap = new mutable.HashMap[String, Set[ConsumerThreadId]]() - for ((topic, nConsumers) <- topicCountMap) { - val consumerSet = new mutable.HashSet[ConsumerThreadId] - assert(nConsumers >= 1) - for (i <- 0 until nConsumers) - consumerSet += ConsumerThreadId(consumerIdString, i) - consumerThreadIdsPerTopicMap.put(topic, consumerSet) - } - consumerThreadIdsPerTopicMap - } - - def constructTopicCount(group: String, consumerId: String, zkUtils: ZkUtils, excludeInternalTopics: Boolean) : TopicCount = { - val dirs = new ZKGroupDirs(group) - val topicCountString = zkUtils.readData(dirs.consumerRegistryDir + "/" + consumerId)._1 - var subscriptionPattern: String = null - var topMap: Map[String, Int] = null - try { - Json.parseFull(topicCountString) match { - case Some(js) => - val consumerRegistrationMap = js.asJsonObject - consumerRegistrationMap.get("pattern") match { - case Some(pattern) => subscriptionPattern = pattern.to[String] - case None => throw new KafkaException("error constructing TopicCount : " + topicCountString) - } - consumerRegistrationMap.get("subscription") match { - case Some(sub) => topMap = sub.to[Map[String, Int]] - case None => throw new KafkaException("error constructing TopicCount : " + topicCountString) - } - case None => throw new KafkaException("error constructing TopicCount : " + topicCountString) - } - } catch { - case e: Throwable => - error("error parsing consumer json string " + topicCountString, e) - throw e - } - - val hasWhiteList = whiteListPattern.equals(subscriptionPattern) - val hasBlackList = blackListPattern.equals(subscriptionPattern) - - if (topMap.isEmpty || !(hasWhiteList || hasBlackList)) { - new StaticTopicCount(consumerId, topMap) - } else { - val regex = topMap.head._1 - val numStreams = topMap.head._2 - val filter = - if (hasWhiteList) - new Whitelist(regex) - else - new Blacklist(regex) - new WildcardTopicCount(zkUtils, consumerId, filter, numStreams, excludeInternalTopics) - } - } - - def constructTopicCount(consumerIdString: String, topicCount: Map[String, Int]) = - new StaticTopicCount(consumerIdString, topicCount) - - def constructTopicCount(consumerIdString: String, filter: TopicFilter, numStreams: Int, zkUtils: ZkUtils, excludeInternalTopics: Boolean) = - new WildcardTopicCount(zkUtils, consumerIdString, filter, numStreams, excludeInternalTopics) - -} - -@deprecated("This class has been deprecated and will be removed in a future release.", "0.11.0.0") -private[kafka] class StaticTopicCount(val consumerIdString: String, - val topicCountMap: Map[String, Int]) - extends TopicCount { - - def getConsumerThreadIdsPerTopic = TopicCount.makeConsumerThreadIdsPerTopic(consumerIdString, topicCountMap) - - def getTopicCountMap = topicCountMap - - def pattern = TopicCount.staticPattern -} - -@deprecated("This class has been deprecated and will be removed in a future release.", "0.11.0.0") -private[kafka] class WildcardTopicCount(zkUtils: ZkUtils, - consumerIdString: String, - topicFilter: TopicFilter, - numStreams: Int, - excludeInternalTopics: Boolean) extends TopicCount { - def getConsumerThreadIdsPerTopic = { - val wildcardTopics = zkUtils.getChildrenParentMayNotExist(ZkUtils.BrokerTopicsPath) - .filter(topic => topicFilter.isTopicAllowed(topic, excludeInternalTopics)) - TopicCount.makeConsumerThreadIdsPerTopic(consumerIdString, Map(wildcardTopics.map((_, numStreams)): _*)) - } - - def getTopicCountMap = Map(CoreUtils.JSONEscapeString(topicFilter.regex) -> numStreams) - - def pattern: String = { - topicFilter match { - case _: Whitelist => TopicCount.whiteListPattern - case _: Blacklist => TopicCount.blackListPattern - } - } - -} - diff --git a/core/src/main/scala/kafka/consumer/TopicEventHandler.scala b/core/src/main/scala/kafka/consumer/TopicEventHandler.scala deleted file mode 100644 index 2852e9bb92800..0000000000000 --- a/core/src/main/scala/kafka/consumer/TopicEventHandler.scala +++ /dev/null @@ -1,25 +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.consumer - -@deprecated("This trait has been deprecated and will be removed in a future release.", "0.11.0.0") -trait TopicEventHandler[T] { - - def handleTopicEvent(allTopics: Seq[T]) - -} diff --git a/core/src/main/scala/kafka/consumer/ZookeeperConsumerConnector.scala b/core/src/main/scala/kafka/consumer/ZookeeperConsumerConnector.scala deleted file mode 100755 index 759da4fe7c070..0000000000000 --- a/core/src/main/scala/kafka/consumer/ZookeeperConsumerConnector.scala +++ /dev/null @@ -1,1045 +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.consumer - -import java.net.InetAddress -import java.util.UUID -import java.util.concurrent._ -import java.util.concurrent.atomic._ -import java.util.concurrent.locks.ReentrantLock - -import com.yammer.metrics.core.Gauge -import kafka.api._ -import kafka.client.ClientUtils -import kafka.cluster._ -import kafka.common._ -import kafka.javaapi.consumer.ConsumerRebalanceListener -import kafka.metrics._ -import kafka.network.BlockingChannel -import kafka.serializer._ -import kafka.utils.CoreUtils.inLock -import kafka.utils.ZkUtils._ -import kafka.utils._ -import org.I0Itec.zkclient.exception.ZkNodeExistsException -import org.I0Itec.zkclient.{IZkChildListener, IZkDataListener, IZkStateListener} -import org.apache.kafka.common.protocol.Errors -import org.apache.kafka.common.security.JaasUtils -import org.apache.kafka.common.utils.Time -import org.apache.zookeeper.Watcher.Event.KeeperState - -import scala.collection._ -import scala.collection.JavaConverters._ - - -/** - * This class handles the consumers interaction with zookeeper - * - * Directories: - * 1. Consumer id registry: - * /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. - * - * The consumer id is picked up from configuration, instead of the sequential id assigned by ZK. Generated sequential - * ids are hard to recover during temporary connection loss to ZK, since it's difficult for the client to figure out - * whether the creation of a sequential znode has succeeded or not. More details can be found at - * (http://wiki.apache.org/hadoop/ZooKeeper/ErrorHandling) - * - * 2. Broker node registry: - * /brokers/[0...N] --> { "host" : "host:port", - * "topics" : {"topic1": ["partition1" ... "partitionN"], ..., - * "topicN": ["partition1" ... "partitionN"] } } - * This is a list of all present broker brokers. A unique logical node id is configured on each broker node. A broker - * node registers itself on start-up and creates a znode with the logical node id under /brokers. The value of the znode - * is a JSON String that contains (1) the host name and the port the broker is listening to, (2) a list of topics that - * the broker serves, (3) a list of logical partitions assigned to each topic on the broker. - * A consumer subscribes to event changes of the broker node registry. - * - * 3. Partition owner registry: - * /consumers/[group_id]/owner/[topic]/[broker_id-partition_id] --> consumer_node_id - * This stores the mapping before broker partitions and consumers. Each partition is owned by a unique consumer - * within a consumer group. The mapping is reestablished after each rebalancing. - * - * 4. Consumer offset tracking: - * /consumers/[group_id]/offsets/[topic]/[broker_id-partition_id] --> offset_counter_value - * Each consumer tracks the offset of the latest message consumed for each partition. - * - */ -@deprecated("This object has been deprecated and will be removed in a future release.", "0.11.0.0") -private[kafka] object ZookeeperConsumerConnector { - val shutdownCommand: FetchedDataChunk = new FetchedDataChunk(null, null, -1L) -} - -@deprecated("This class has been deprecated and will be removed in a future release.", "0.11.0.0") -private[kafka] class ZookeeperConsumerConnector(val config: ConsumerConfig, - val enableFetcher: Boolean) // for testing only - extends ConsumerConnector with Logging with KafkaMetricsGroup { - - private val isShuttingDown = new AtomicBoolean(false) - private val rebalanceLock = new Object - private var fetcher: Option[ConsumerFetcherManager] = None - private var zkUtils: ZkUtils = 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]] - private val scheduler = new KafkaScheduler(threads = 1, threadNamePrefix = "kafka-consumer-scheduler-") - private val messageStreamCreated = new AtomicBoolean(false) - - private var sessionExpirationListener: ZKSessionExpireListener = null - private var topicPartitionChangeListener: ZKTopicPartitionChangeListener = null - private var loadBalancerListener: ZKRebalancerListener = null - - private var offsetsChannel: BlockingChannel = null - private val offsetsChannelLock = new Object - - private var wildcardTopicWatcher: ZookeeperTopicEventWatcher = null - private var consumerRebalanceListener: ConsumerRebalanceListener = null - - // useful for tracking migration of consumers to store offsets in kafka - private val kafkaCommitMeter = newMeter("KafkaCommitsPerSec", "commits", TimeUnit.SECONDS, Map("clientId" -> config.clientId)) - private val zkCommitMeter = newMeter("ZooKeeperCommitsPerSec", "commits", TimeUnit.SECONDS, Map("clientId" -> config.clientId)) - private val rebalanceTimer = new KafkaTimer(newTimer("RebalanceRateAndTime", TimeUnit.MILLISECONDS, TimeUnit.SECONDS, Map("clientId" -> config.clientId))) - - newGauge( - "yammer-metrics-count", - new Gauge[Int] { - def value = { - com.yammer.metrics.Metrics.defaultRegistry().allMetrics().size() - } - } - ) - - val consumerIdString = { - var consumerUuid : String = null - config.consumerId match { - case Some(consumerId) // for testing only - => consumerUuid = consumerId - case None // generate unique consumerId automatically - => val uuid = UUID.randomUUID() - consumerUuid = "%s-%d-%s".format( - InetAddress.getLocalHost.getHostName, System.currentTimeMillis, - uuid.getMostSignificantBits().toHexString.substring(0,8)) - } - config.groupId + "_" + consumerUuid - } - this.logIdent = "[" + consumerIdString + "], " - - connectZk() - createFetcher() - ensureOffsetManagerConnected() - - if (config.autoCommitEnable) { - scheduler.startup - info("starting auto committer every " + config.autoCommitIntervalMs + " ms") - scheduler.schedule("kafka-consumer-autocommit", - autoCommit _, - delay = config.autoCommitIntervalMs, - period = config.autoCommitIntervalMs, - unit = TimeUnit.MILLISECONDS) - } - - KafkaMetricsReporter.startReporters(config.props) - AppInfo.registerInfo() - - def this(config: ConsumerConfig) = this(config, true) - - def createMessageStreams(topicCountMap: Map[String,Int]): Map[String, List[KafkaStream[Array[Byte],Array[Byte]]]] = - createMessageStreams(topicCountMap, new DefaultDecoder(), new DefaultDecoder()) - - def createMessageStreams[K,V](topicCountMap: Map[String,Int], keyDecoder: Decoder[K], valueDecoder: Decoder[V]) - : Map[String, List[KafkaStream[K,V]]] = { - if (messageStreamCreated.getAndSet(true)) - throw new MessageStreamsExistException(this.getClass.getSimpleName + - " can create message streams at most once",null) - consume(topicCountMap, keyDecoder, valueDecoder) - } - - def createMessageStreamsByFilter[K,V](topicFilter: TopicFilter, - numStreams: Int, - keyDecoder: Decoder[K] = new DefaultDecoder(), - valueDecoder: Decoder[V] = new DefaultDecoder()) = { - val wildcardStreamsHandler = new WildcardStreamsHandler[K,V](topicFilter, numStreams, keyDecoder, valueDecoder) - wildcardStreamsHandler.streams - } - - def setConsumerRebalanceListener(listener: ConsumerRebalanceListener) { - if (messageStreamCreated.get()) - throw new MessageStreamsExistException(this.getClass.getSimpleName + - " can only set consumer rebalance listener before creating streams",null) - consumerRebalanceListener = listener - } - - private def createFetcher() { - if (enableFetcher) - fetcher = Some(new ConsumerFetcherManager(consumerIdString, config, zkUtils)) - } - - private def connectZk() { - info("Connecting to zookeeper instance at " + config.zkConnect) - zkUtils = ZkUtils(config.zkConnect, - config.zkSessionTimeoutMs, - config.zkConnectionTimeoutMs, - JaasUtils.isZkSecurityEnabled()) - } - - // 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, zkUtils, - config.offsetsChannelSocketTimeoutMs, config.offsetsChannelBackoffMs) - - debug("Connected to offset manager %s:%d.".format(offsetsChannel.host, offsetsChannel.port)) - } - } - - def shutdown() { - val canShutdown = isShuttingDown.compareAndSet(false, true) - if (canShutdown) { - info("ZKConsumerConnector shutting down") - val startTime = System.nanoTime() - KafkaMetricsGroup.removeAllConsumerMetrics(config.clientId) - if (wildcardTopicWatcher != null) - wildcardTopicWatcher.shutdown() - rebalanceLock synchronized { - try { - if (config.autoCommitEnable) - scheduler.shutdown() - fetcher.foreach(_.stopConnections()) - sendShutdownToAllQueues() - if (config.autoCommitEnable) - commitOffsets(true) - if (zkUtils != null) { - zkUtils.close() - zkUtils = null - } - - if (offsetsChannel != null) offsetsChannel.disconnect() - } catch { - case e: Throwable => - fatal("error during consumer connector shutdown", e) - } - info("ZKConsumerConnector shutdown completed in " + (System.nanoTime() - startTime) / 1000000 + " ms") - } - } - } - - def consume[K, V](topicCountMap: scala.collection.Map[String,Int], keyDecoder: Decoder[K], valueDecoder: Decoder[V]) - : Map[String,List[KafkaStream[K,V]]] = { - debug("entering consume ") - if (topicCountMap == null) - throw new RuntimeException("topicCountMap is null") - - val topicCount = TopicCount.constructTopicCount(consumerIdString, topicCountMap) - - val topicThreadIds = topicCount.getConsumerThreadIdsPerTopic - - // make a list of (queue,stream) pairs, one pair for each threadId - val queuesAndStreams = topicThreadIds.values.map(threadIdSet => - threadIdSet.map(_ => { - val queue = new LinkedBlockingQueue[FetchedDataChunk](config.queuedMaxMessages) - val stream = new KafkaStream[K,V]( - queue, config.consumerTimeoutMs, keyDecoder, valueDecoder, config.clientId) - (queue, stream) - }) - ).flatten.toList - - val dirs = new ZKGroupDirs(config.groupId) - registerConsumerInZK(dirs, consumerIdString, topicCount) - reinitializeConsumer(topicCount, queuesAndStreams) - - loadBalancerListener.kafkaMessageAndMetadataStreams.asInstanceOf[Map[String, List[KafkaStream[K,V]]]] - } - - // this API is used by unit tests only - def getTopicRegistry: Pool[String, Pool[Int, PartitionTopicInfo]] = topicRegistry - - private def registerConsumerInZK(dirs: ZKGroupDirs, consumerIdString: String, topicCount: TopicCount) { - info("begin registering consumer " + consumerIdString + " in ZK") - val timestamp = Time.SYSTEM.milliseconds.toString - - val consumerRegistrationInfo = Json.encodeAsString(Map("version" -> 1, - "subscription" -> topicCount.getTopicCountMap.asJava, - "pattern" -> topicCount.pattern, - "timestamp" -> timestamp - ).asJava) - - val zkWatchedEphemeral = new ZKCheckedEphemeral(dirs. - consumerRegistryDir + "/" + consumerIdString, - consumerRegistrationInfo, - zkUtils.zkConnection.getZookeeper, - false) - zkWatchedEphemeral.create() - - info("end registering consumer " + consumerIdString + " in ZK") - } - - private def sendShutdownToAllQueues() = { - for (queue <- topicThreadIdAndQueues.values.toSet[BlockingQueue[FetchedDataChunk]]) { - debug("Clearing up queue") - queue.clear() - queue.put(ZookeeperConsumerConnector.shutdownCommand) - debug("Cleared queue and sent shutdown command") - } - } - - def autoCommit() { - trace("auto committing") - try { - commitOffsets(isAutoCommit = false) - } - catch { - case t: Throwable => - // log it and let it go - error("exception during autoCommit: ", t) - } - } - - def commitOffsetToZooKeeper(topicPartition: TopicAndPartition, offset: Long) { - if (checkpointedZkOffsets.get(topicPartition) != offset) { - val topicDirs = new ZKGroupTopicDirs(config.groupId, topicPartition.topic) - zkUtils.updatePersistentPath(topicDirs.consumerOffsetDir + "/" + topicPartition.partition, offset.toString) - checkpointedZkOffsets.put(topicPartition, offset) - zkCommitMeter.mark() - } - } - - /** - * KAFKA-1743: This method added for backward compatibility. - */ - def commitOffsets { commitOffsets(true) } - - def commitOffsets(isAutoCommit: Boolean) { - - val offsetsToCommit = - immutable.Map(topicRegistry.values.flatMap { partitionTopicInfos => - partitionTopicInfos.values.map { info => - TopicAndPartition(info.topic, info.partitionId) -> OffsetAndMetadata(info.getConsumeOffset()) - } - }.toSeq: _*) - - commitOffsets(offsetsToCommit, isAutoCommit) - - } - - def commitOffsets(offsetsToCommit: immutable.Map[TopicAndPartition, OffsetAndMetadata], isAutoCommit: Boolean) { - trace("OffsetMap: %s".format(offsetsToCommit)) - var retriesRemaining = 1 + (if (isAutoCommit) 0 else config.offsetsCommitMaxRetries) // no retries for commits from auto-commit - var done = false - while (!done) { - val committed = offsetsChannelLock synchronized { - // committed when we receive either no error codes or only MetadataTooLarge errors - if (offsetsToCommit.size > 0) { - if (config.offsetsStorage == "zookeeper") { - offsetsToCommit.foreach { case (topicAndPartition, offsetAndMetadata) => - commitOffsetToZooKeeper(topicAndPartition, offsetAndMetadata.offset) - } - true - } else { - val offsetCommitRequest = OffsetCommitRequest(config.groupId, offsetsToCommit, clientId = config.clientId) - ensureOffsetManagerConnected() - try { - kafkaCommitMeter.mark(offsetsToCommit.size) - offsetsChannel.send(offsetCommitRequest) - val offsetCommitResponse = OffsetCommitResponse.readFrom(offsetsChannel.receive().payload()) - trace("Offset commit response: %s.".format(offsetCommitResponse)) - - val (commitFailed, retryableIfFailed, shouldRefreshCoordinator, errorCount) = { - offsetCommitResponse.commitStatus.foldLeft(false, false, false, 0) { case (folded, (topicPartition, error)) => - - if (error == Errors.NONE && config.dualCommitEnabled) { - val offset = offsetsToCommit(topicPartition).offset - commitOffsetToZooKeeper(topicPartition, offset) - } - - (folded._1 || // update commitFailed - error != Errors.NONE, - - folded._2 || // update retryableIfFailed - (only metadata too large is not retryable) - (error != Errors.NONE && error != Errors.OFFSET_METADATA_TOO_LARGE), - - folded._3 || // update shouldRefreshCoordinator - error == Errors.NOT_COORDINATOR || - error == Errors.COORDINATOR_NOT_AVAILABLE, - - // update error count - folded._4 + (if (error != Errors.NONE) 1 else 0)) - } - } - debug(errorCount + " errors in offset commit response.") - - - if (shouldRefreshCoordinator) { - debug("Could not commit offsets (because offset coordinator has moved or is unavailable).") - offsetsChannel.disconnect() - } - - if (commitFailed && retryableIfFailed) - false - else - true - } - catch { - case t: Throwable => - error("Error while committing offsets.", t) - offsetsChannel.disconnect() - false - } - } - } else { - debug("No updates to offsets since last commit.") - true - } - } - - done = { - retriesRemaining -= 1 - retriesRemaining == 0 || committed - } - - if (!done) { - debug("Retrying offset commit in %d ms".format(config.offsetsChannelBackoffMs)) - Thread.sleep(config.offsetsChannelBackoffMs) - } - } - } - - private def fetchOffsetFromZooKeeper(topicPartition: TopicAndPartition) = { - val dirs = new ZKGroupTopicDirs(config.groupId, topicPartition.topic) - val offsetString = zkUtils.readDataMaybeNull(dirs.consumerOffsetDir + "/" + topicPartition.partition)._1 - offsetString match { - case Some(offsetStr) => (topicPartition, OffsetMetadataAndError(offsetStr.toLong)) - case None => (topicPartition, OffsetMetadataAndError.NoOffset) - } - } - - private def fetchOffsets(partitions: Seq[TopicAndPartition]) = { - if (partitions.isEmpty) - Some(OffsetFetchResponse(Map.empty)) - else if (config.offsetsStorage == "zookeeper") { - val offsets = partitions.map(fetchOffsetFromZooKeeper) - Some(OffsetFetchResponse(immutable.Map(offsets:_*))) - } else { - val offsetFetchRequest = OffsetFetchRequest(groupId = config.groupId, requestInfo = partitions, clientId = config.clientId) - - var offsetFetchResponseOpt: Option[OffsetFetchResponse] = None - while (!isShuttingDown.get && !offsetFetchResponseOpt.isDefined) { - offsetFetchResponseOpt = offsetsChannelLock synchronized { - ensureOffsetManagerConnected() - try { - offsetsChannel.send(offsetFetchRequest) - val offsetFetchResponse = OffsetFetchResponse.readFrom(offsetsChannel.receive().payload()) - trace("Offset fetch response: %s.".format(offsetFetchResponse)) - - val (leaderChanged, loadInProgress) = - offsetFetchResponse.requestInfo.values.foldLeft(false, false) { case (folded, offsetMetadataAndError) => - (folded._1 || (offsetMetadataAndError.error == Errors.NOT_COORDINATOR), - folded._2 || (offsetMetadataAndError.error == Errors.COORDINATOR_LOAD_IN_PROGRESS)) - } - - if (leaderChanged) { - offsetsChannel.disconnect() - debug("Could not fetch offsets (because offset manager has moved).") - None // retry - } - else if (loadInProgress) { - debug("Could not fetch offsets (because offset cache is being loaded).") - None // retry - } - else { - if (config.dualCommitEnabled) { - // if dual-commit is enabled (i.e., if a consumer group is migrating offsets to kafka), then pick the - // maximum between offsets in zookeeper and kafka. - val kafkaOffsets = offsetFetchResponse.requestInfo - val mostRecentOffsets = kafkaOffsets.map { case (topicPartition, kafkaOffset) => - val zkOffset = fetchOffsetFromZooKeeper(topicPartition)._2.offset - val mostRecentOffset = zkOffset.max(kafkaOffset.offset) - (topicPartition, OffsetMetadataAndError(mostRecentOffset, kafkaOffset.metadata, Errors.NONE)) - } - Some(OffsetFetchResponse(mostRecentOffsets)) - } - else - Some(offsetFetchResponse) - } - } - catch { - case e: Exception => - warn("Error while fetching offsets from %s:%d. Possible cause: %s".format(offsetsChannel.host, offsetsChannel.port, e.getMessage)) - offsetsChannel.disconnect() - None // retry - } - } - - if (offsetFetchResponseOpt.isEmpty) { - debug("Retrying offset fetch in %d ms".format(config.offsetsChannelBackoffMs)) - Thread.sleep(config.offsetsChannelBackoffMs) - } - } - - offsetFetchResponseOpt - } - } - - - class ZKSessionExpireListener(val dirs: ZKGroupDirs, - val consumerIdString: String, - val topicCount: TopicCount, - val loadBalancerListener: ZKRebalancerListener) - extends IZkStateListener { - @throws[Exception] - def handleStateChanged(state: KeeperState) { - // do nothing, since zkclient will do reconnect for us. - } - - /** - * Called after the zookeeper session has expired and a new session has been created. You would have to re-create - * any ephemeral nodes here. - * - * @throws Exception - * On any error. - */ - @throws[Exception] - def handleNewSession() { - /** - * When we get a SessionExpired event, we lost all ephemeral nodes and zkclient has reestablished a - * connection for us. We need to release the ownership of the current consumer and re-register this - * consumer in the consumer registry and trigger a rebalance. - */ - info("ZK expired; release old broker parition ownership; re-register consumer " + consumerIdString) - loadBalancerListener.resetState() - registerConsumerInZK(dirs, consumerIdString, topicCount) - // explicitly trigger load balancing for this consumer - loadBalancerListener.syncedRebalance() - // There is no need to resubscribe to child and state changes. - // The child change watchers will be set inside rebalance when we read the children list. - } - - override def handleSessionEstablishmentError(error: Throwable): Unit = { - fatal("Could not establish session with zookeeper", error) - } - } - - class ZKTopicPartitionChangeListener(val loadBalancerListener: ZKRebalancerListener) - extends IZkDataListener { - - def handleDataChange(dataPath : String, data: Object) { - try { - info("Topic info for path " + dataPath + " changed to " + data.toString + ", triggering rebalance") - // queue up the rebalance event - loadBalancerListener.rebalanceEventTriggered() - // There is no need to re-subscribe the watcher since it will be automatically - // re-registered upon firing of this event by zkClient - } catch { - case e: Throwable => error("Error while handling topic partition change for data path " + dataPath, e ) - } - } - - @throws[Exception] - def handleDataDeleted(dataPath : String) { - // TODO: This need to be implemented when we support delete topic - warn("Topic for path " + dataPath + " gets deleted, which should not happen at this time") - } - } - - class ZKRebalancerListener(val group: String, val consumerIdString: String, - val kafkaMessageAndMetadataStreams: mutable.Map[String,List[KafkaStream[_,_]]]) - extends IZkChildListener { - - private val partitionAssignor = PartitionAssignor.createInstance(config.partitionAssignmentStrategy) - - private var isWatcherTriggered = false - private val lock = new ReentrantLock - private val cond = lock.newCondition() - - @volatile private var allTopicsOwnedPartitionsCount = 0 - newGauge("OwnedPartitionsCount", - new Gauge[Int] { - def value() = allTopicsOwnedPartitionsCount - }, - Map("clientId" -> config.clientId, "groupId" -> config.groupId)) - - private def ownedPartitionsCountMetricTags(topic: String) = Map("clientId" -> config.clientId, "groupId" -> config.groupId, "topic" -> topic) - - private val watcherExecutorThread = new Thread(consumerIdString + "_watcher_executor") { - override def run() { - info("starting watcher executor thread for consumer " + consumerIdString) - var doRebalance = false - while (!isShuttingDown.get) { - try { - lock.lock() - try { - if (!isWatcherTriggered) - cond.await(1000, TimeUnit.MILLISECONDS) // wake up periodically so that it can check the shutdown flag - } finally { - doRebalance = isWatcherTriggered - isWatcherTriggered = false - lock.unlock() - } - if (doRebalance) - syncedRebalance - } catch { - case t: Throwable => error("error during syncedRebalance", t) - } - } - info("stopping watcher executor thread for consumer " + consumerIdString) - } - } - watcherExecutorThread.start() - - @throws[Exception] - def handleChildChange(parentPath : String, curChilds : java.util.List[String]) { - rebalanceEventTriggered() - } - - def rebalanceEventTriggered() { - inLock(lock) { - isWatcherTriggered = true - cond.signalAll() - } - } - - private def deletePartitionOwnershipFromZK(topic: String, partition: Int) { - val topicDirs = new ZKGroupTopicDirs(group, topic) - val znode = topicDirs.consumerOwnerDir + "/" + partition - zkUtils.deletePath(znode) - debug("Consumer " + consumerIdString + " releasing " + znode) - } - - private def releasePartitionOwnership(localTopicRegistry: Pool[String, Pool[Int, PartitionTopicInfo]])= { - info("Releasing partition ownership") - for ((topic, infos) <- localTopicRegistry) { - for(partition <- infos.keys) { - deletePartitionOwnershipFromZK(topic, partition) - } - removeMetric("OwnedPartitionsCount", ownedPartitionsCountMetricTags(topic)) - localTopicRegistry.remove(topic) - } - allTopicsOwnedPartitionsCount = 0 - } - - def resetState() { - topicRegistry.clear - } - - def syncedRebalance() { - rebalanceLock synchronized { - rebalanceTimer.time { - for (i <- 0 until config.rebalanceMaxRetries) { - if(isShuttingDown.get()) { - return - } - info("begin rebalancing consumer " + consumerIdString + " try #" + i) - var done = false - var cluster: Cluster = null - try { - cluster = zkUtils.getCluster() - 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) - } - } - } - - throw new ConsumerRebalanceFailedException(consumerIdString + " can't rebalance after " + config.rebalanceMaxRetries +" retries") - } - - private def rebalance(cluster: Cluster): Boolean = { - val myTopicThreadIdsMap = TopicCount.constructTopicCount( - group, consumerIdString, zkUtils, config.excludeInternalTopics).getConsumerThreadIdsPerTopic - val brokers = zkUtils.getAllBrokersInCluster() - if (brokers.size == 0) { - // This can happen in a rare case when there are no brokers available in the cluster when the consumer is started. - // We log a warning and register for child changes on brokers/id so that rebalance can be triggered when the brokers - // are up. - warn("no brokers found when trying to rebalance.") - zkUtils.subscribeChildChanges(BrokerIdsPath, loadBalancerListener) - true - } - else { - /** - * fetchers must be stopped to avoid data duplication, since if the current - * rebalancing attempt fails, the partitions that are released could be owned by another consumer. - * But if we don't stop the fetchers first, this consumer would continue returning data for released - * partitions in parallel. So, not stopping the fetchers leads to duplicate data. - */ - closeFetchers(cluster, kafkaMessageAndMetadataStreams, myTopicThreadIdsMap) - 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 - topicRegistry.map(topics => - topics._1 -> topics._2.keys // note this is incorrect, see KAFKA-2284 - ).toMap.asJava.asInstanceOf[java.util.Map[String, java.util.Set[java.lang.Integer]]] - ) - } - releasePartitionOwnership(topicRegistry) - val assignmentContext = new AssignmentContext(group, consumerIdString, config.excludeInternalTopics, zkUtils) - val globalPartitionAssignment = partitionAssignor.assign(assignmentContext) - val partitionAssignment = globalPartitionAssignment.get(assignmentContext.consumerId) - val currentTopicRegistry = new Pool[String, Pool[Int, PartitionTopicInfo]]( - valueFactory = Some((_: String) => new Pool[Int, PartitionTopicInfo])) - - // fetch current offsets for all topic-partitions - val topicPartitions = partitionAssignment.keySet.toSeq - - val offsetFetchResponseOpt = fetchOffsets(topicPartitions) - - if (isShuttingDown.get || !offsetFetchResponseOpt.isDefined) - false - else { - val offsetFetchResponse = offsetFetchResponseOpt.get - topicPartitions.foreach { case tp@ TopicAndPartition(topic, partition) => - val offset = offsetFetchResponse.requestInfo(tp).offset - val threadId = partitionAssignment(tp) - addPartitionTopicInfo(currentTopicRegistry, partition, topic, offset, threadId) - } - - /** - * move the partition ownership here, since that can be used to indicate a truly successful re-balancing attempt - * A rebalancing attempt is completed successfully only after the fetchers have been started correctly - */ - if(reflectPartitionOwnershipDecision(partitionAssignment)) { - allTopicsOwnedPartitionsCount = partitionAssignment.size - - partitionAssignment.view.groupBy { case (topicPartition, _) => topicPartition.topic } - .foreach { case (topic, partitionThreadPairs) => - 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: Integer) -> consumerThreadId - }).toMap - topic -> partitionOwnershipForTopicScalaMap.asJava - }) - consumerRebalanceListener.beforeStartingFetchers( - consumerIdString, - partitionAssigmentMapForCallback.asJava - ) - } - updateFetcher(cluster) - true - } else { - false - } - } - } - } - - private def closeFetchersForQueues(cluster: Cluster, - messageStreams: Map[String,List[KafkaStream[_,_]]], - queuesToBeCleared: Iterable[BlockingQueue[FetchedDataChunk]]) { - val allPartitionInfos = topicRegistry.values.map(p => p.values).flatten - fetcher.foreach { f => - f.stopConnections() - clearFetcherQueues(allPartitionInfos, cluster, queuesToBeCleared, messageStreams) - /** - * 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 - * call will ensure that the offsets committed now will be used by the next consumer thread owning the partition - * for the current data chunk. Since the fetchers are already shutdown and this is the last chunk to be iterated - * 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) { - info("Committing all offsets after clearing the fetcher queues") - commitOffsets(true) - } - } - } - - private def clearFetcherQueues(topicInfos: Iterable[PartitionTopicInfo], cluster: Cluster, - queuesTobeCleared: Iterable[BlockingQueue[FetchedDataChunk]], - messageStreams: Map[String,List[KafkaStream[_,_]]]) { - - // Clear all but the currently iterated upon chunk in the consumer thread's queue - queuesTobeCleared.foreach(_.clear) - info("Cleared all relevant queues for this fetcher") - - // Also clear the currently iterated upon chunk in the consumer threads - if(messageStreams != null) - messageStreams.foreach(_._2.foreach(s => s.clear())) - - info("Cleared the data chunks in all the consumer message iterators") - - } - - private def closeFetchers(cluster: Cluster, messageStreams: Map[String,List[KafkaStream[_,_]]], - relevantTopicThreadIdsMap: Map[String, Set[ConsumerThreadId]]) { - // only clear the fetcher queues for certain topic partitions that *might* no longer be served by this consumer - // after this rebalancing attempt - val queuesTobeCleared = topicThreadIdAndQueues.filter(q => relevantTopicThreadIdsMap.contains(q._1._1)).map(q => q._2) - closeFetchersForQueues(cluster, messageStreams, queuesTobeCleared) - } - - private def updateFetcher(cluster: Cluster) { - // update partitions for fetcher - var allPartitionInfos : List[PartitionTopicInfo] = Nil - for (partitionInfos <- topicRegistry.values) - for (partition <- partitionInfos.values) - allPartitionInfos ::= partition - info("Consumer " + consumerIdString + " selected partitions : " + - allPartitionInfos.sortWith((s,t) => s.partitionId < t.partitionId).map(_.toString).mkString(",")) - - fetcher.foreach(_.startConnections(allPartitionInfos, cluster)) - } - - private def reflectPartitionOwnershipDecision(partitionAssignment: Map[TopicAndPartition, ConsumerThreadId]): Boolean = { - var successfullyOwnedPartitions : List[(String, Int)] = Nil - val partitionOwnershipSuccessful = partitionAssignment.map { partitionOwner => - val topic = partitionOwner._1.topic - val partition = partitionOwner._1.partition - val consumerThreadId = partitionOwner._2 - val partitionOwnerPath = zkUtils.getConsumerPartitionOwnerPath(group, topic, partition) - try { - zkUtils.createEphemeralPathExpectConflict(partitionOwnerPath, consumerThreadId.toString) - info(consumerThreadId + " successfully owned partition " + partition + " for topic " + topic) - successfullyOwnedPartitions ::= (topic, partition) - true - } catch { - case _: ZkNodeExistsException => - // The node hasn't been deleted by the original owner. So wait a bit and retry. - info("waiting for the partition ownership to be deleted: " + partition + " for topic " + topic) - false - } - } - val hasPartitionOwnershipFailed = partitionOwnershipSuccessful.foldLeft(0)((sum, decision) => sum + (if(decision) 0 else 1)) - /* even if one of the partition ownership attempt has failed, return false */ - if(hasPartitionOwnershipFailed > 0) { - // remove all paths that we have owned in ZK - successfullyOwnedPartitions.foreach(topicAndPartition => deletePartitionOwnershipFromZK(topicAndPartition._1, topicAndPartition._2)) - false - } - else true - } - - private def addPartitionTopicInfo(currentTopicRegistry: Pool[String, Pool[Int, PartitionTopicInfo]], - partition: Int, topic: String, - offset: Long, consumerThreadId: ConsumerThreadId) { - val partTopicInfoMap = currentTopicRegistry.getAndMaybePut(topic) - - val queue = topicThreadIdAndQueues.get((topic, consumerThreadId)) - val consumedOffset = new AtomicLong(offset) - val fetchedOffset = new AtomicLong(offset) - val partTopicInfo = new PartitionTopicInfo(topic, - partition, - queue, - consumedOffset, - fetchedOffset, - new AtomicInteger(config.fetchMessageMaxBytes), - config.clientId) - partTopicInfoMap.put(partition, partTopicInfo) - debug(partTopicInfo + " selected new offset " + offset) - checkpointedZkOffsets.put(TopicAndPartition(topic, partition), offset) - } - } - - private def reinitializeConsumer[K,V]( - topicCount: TopicCount, - queuesAndStreams: List[(LinkedBlockingQueue[FetchedDataChunk],KafkaStream[K,V])]) { - - val dirs = new ZKGroupDirs(config.groupId) - - // listener to consumer and partition changes - if (loadBalancerListener == null) { - val topicStreamsMap = new mutable.HashMap[String,List[KafkaStream[K,V]]] - loadBalancerListener = new ZKRebalancerListener( - config.groupId, consumerIdString, topicStreamsMap.asInstanceOf[scala.collection.mutable.Map[String, List[KafkaStream[_,_]]]]) - } - - // create listener for session expired event if not exist yet - if (sessionExpirationListener == null) - sessionExpirationListener = new ZKSessionExpireListener( - dirs, consumerIdString, topicCount, loadBalancerListener) - - // create listener for topic partition change event if not exist yet - if (topicPartitionChangeListener == null) - topicPartitionChangeListener = new ZKTopicPartitionChangeListener(loadBalancerListener) - - val topicStreamsMap = loadBalancerListener.kafkaMessageAndMetadataStreams - - // map of {topic -> Set(thread-1, thread-2, ...)} - val consumerThreadIdsPerTopic: Map[String, Set[ConsumerThreadId]] = - topicCount.getConsumerThreadIdsPerTopic - - val allQueuesAndStreams = topicCount match { - case _: WildcardTopicCount => - /* - * Wild-card consumption streams share the same queues, so we need to - * duplicate the list for the subsequent zip operation. - */ - (1 to consumerThreadIdsPerTopic.keySet.size).flatMap(_ => queuesAndStreams).toList - case _: StaticTopicCount => - queuesAndStreams - } - - val topicThreadIds = consumerThreadIdsPerTopic.map { case (topic, threadIds) => - threadIds.map((topic, _)) - }.flatten - - require(topicThreadIds.size == allQueuesAndStreams.size, - "Mismatch between thread ID count (%d) and queue count (%d)" - .format(topicThreadIds.size, allQueuesAndStreams.size)) - val threadQueueStreamPairs = topicThreadIds.zip(allQueuesAndStreams) - - threadQueueStreamPairs.foreach(e => { - val topicThreadId = e._1 - val q = e._2._1 - topicThreadIdAndQueues.put(topicThreadId, q) - debug("Adding topicThreadId %s and queue %s to topicThreadIdAndQueues data structure".format(topicThreadId, q.toString)) - newGauge( - "FetchQueueSize", - new Gauge[Int] { - def value = q.size - }, - Map("clientId" -> config.clientId, - "topic" -> topicThreadId._1, - "threadId" -> topicThreadId._2.threadId.toString) - ) - }) - - val groupedByTopic = threadQueueStreamPairs.groupBy(_._1._1) - groupedByTopic.foreach(e => { - val topic = e._1 - val streams = e._2.map(_._2._2).toList - topicStreamsMap += (topic -> streams) - debug("adding topic %s and %d streams to map.".format(topic, streams.size)) - }) - - // listener to consumer and partition changes - zkUtils.subscribeStateChanges(sessionExpirationListener) - - zkUtils.subscribeChildChanges(dirs.consumerRegistryDir, loadBalancerListener) - - topicStreamsMap.foreach { topicAndStreams => - // register on broker partition path changes - val topicPath = BrokerTopicsPath + "/" + topicAndStreams._1 - zkUtils.subscribeDataChanges(topicPath, topicPartitionChangeListener) - } - - // explicitly trigger load balancing for this consumer - loadBalancerListener.syncedRebalance() - } - - class WildcardStreamsHandler[K,V](topicFilter: TopicFilter, - numStreams: Int, - keyDecoder: Decoder[K], - valueDecoder: Decoder[V]) - extends TopicEventHandler[String] { - - if (messageStreamCreated.getAndSet(true)) - throw new RuntimeException("Each consumer connector can create " + - "message streams by filter at most once.") - - private val wildcardQueuesAndStreams = (1 to numStreams) - .map(_ => { - val queue = new LinkedBlockingQueue[FetchedDataChunk](config.queuedMaxMessages) - val stream = new KafkaStream[K,V](queue, - config.consumerTimeoutMs, - keyDecoder, - valueDecoder, - config.clientId) - (queue, stream) - }).toList - - // bootstrap with existing topics - private var wildcardTopics = - zkUtils.getChildrenParentMayNotExist(BrokerTopicsPath) - .filter(topic => topicFilter.isTopicAllowed(topic, config.excludeInternalTopics)) - - private val wildcardTopicCount = TopicCount.constructTopicCount( - consumerIdString, topicFilter, numStreams, zkUtils, config.excludeInternalTopics) - - val dirs = new ZKGroupDirs(config.groupId) - registerConsumerInZK(dirs, consumerIdString, wildcardTopicCount) - reinitializeConsumer(wildcardTopicCount, wildcardQueuesAndStreams) - - /* - * Topic events will trigger subsequent synced rebalances. - */ - info("Creating topic event watcher for topics " + topicFilter) - wildcardTopicWatcher = new ZookeeperTopicEventWatcher(zkUtils, this) - - def handleTopicEvent(allTopics: Seq[String]) { - debug("Handling topic event") - - val updatedTopics = allTopics.filter(topic => topicFilter.isTopicAllowed(topic, config.excludeInternalTopics)) - - val addedTopics = updatedTopics filterNot (wildcardTopics contains) - if (addedTopics.nonEmpty) - info("Topic event: added topics = %s" - .format(addedTopics)) - - /* - * TODO: Deleted topics are interesting (and will not be a concern until - * 0.8 release). We may need to remove these topics from the rebalance - * listener's map in reinitializeConsumer. - */ - val deletedTopics = wildcardTopics filterNot (updatedTopics contains) - if (deletedTopics.nonEmpty) - info("Topic event: deleted topics = %s" - .format(deletedTopics)) - - wildcardTopics = updatedTopics - info("Topics to consume = %s".format(wildcardTopics)) - - if (addedTopics.nonEmpty || deletedTopics.nonEmpty) - reinitializeConsumer(wildcardTopicCount, wildcardQueuesAndStreams) - } - - def streams: Seq[KafkaStream[K,V]] = - wildcardQueuesAndStreams.map(_._2) - } -} diff --git a/core/src/main/scala/kafka/consumer/ZookeeperTopicEventWatcher.scala b/core/src/main/scala/kafka/consumer/ZookeeperTopicEventWatcher.scala deleted file mode 100644 index 8ce204e3b9114..0000000000000 --- a/core/src/main/scala/kafka/consumer/ZookeeperTopicEventWatcher.scala +++ /dev/null @@ -1,103 +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.consumer - -import scala.collection.JavaConverters._ -import kafka.utils.{ZkUtils, Logging} -import org.I0Itec.zkclient.{IZkStateListener, IZkChildListener} -import org.apache.zookeeper.Watcher.Event.KeeperState - -@deprecated("This class has been deprecated and will be removed in a future release.", "0.11.0.0") -class ZookeeperTopicEventWatcher(val zkUtils: ZkUtils, - val eventHandler: TopicEventHandler[String]) extends Logging { - - val lock = new Object() - - startWatchingTopicEvents() - - private def startWatchingTopicEvents() { - val topicEventListener = new ZkTopicEventListener() - zkUtils.makeSurePersistentPathExists(ZkUtils.BrokerTopicsPath) - - zkUtils.subscribeStateChanges(new ZkSessionExpireListener(topicEventListener)) - - val topics = zkUtils.subscribeChildChanges(ZkUtils.BrokerTopicsPath, topicEventListener).getOrElse { - throw new AssertionError(s"Expected ${ZkUtils.BrokerTopicsPath} to exist, but it does not. ") - } - - // call to bootstrap topic list - topicEventListener.handleChildChange(ZkUtils.BrokerTopicsPath, topics.asJava) - } - - private def stopWatchingTopicEvents() { zkUtils.unsubscribeAll() } - - def shutdown() { - lock.synchronized { - info("Shutting down topic event watcher.") - if (zkUtils != null) { - stopWatchingTopicEvents() - } - else { - warn("Cannot shutdown since the embedded zookeeper client has already closed.") - } - } - } - - class ZkTopicEventListener extends IZkChildListener { - - @throws[Exception] - def handleChildChange(parent: String, children: java.util.List[String]) { - lock.synchronized { - try { - if (zkUtils != null) { - val latestTopics = zkUtils.getChildren(ZkUtils.BrokerTopicsPath) - debug("all topics: %s".format(latestTopics)) - eventHandler.handleTopicEvent(latestTopics) - } - } - catch { - case e: Throwable => - error("error in handling child changes", e) - } - } - } - - } - - class ZkSessionExpireListener(val topicEventListener: ZkTopicEventListener) - extends IZkStateListener { - - @throws[Exception] - def handleStateChanged(state: KeeperState) { } - - @throws[Exception] - def handleNewSession() { - lock.synchronized { - if (zkUtils != null) { - info("ZK expired: resubscribing topic event listener to topic registry") - zkUtils.subscribeChildChanges(ZkUtils.BrokerTopicsPath, topicEventListener) - } - } - } - - override def handleSessionEstablishmentError(error: Throwable): Unit = { - //no-op ZookeeperConsumerConnector should log error. - } - } -} - diff --git a/core/src/main/scala/kafka/consumer/package.html b/core/src/main/scala/kafka/consumer/package.html deleted file mode 100644 index 9e06da8f0c0ef..0000000000000 --- a/core/src/main/scala/kafka/consumer/package.html +++ /dev/null @@ -1,19 +0,0 @@ - -This is the consumer API for kafka. \ No newline at end of file diff --git a/core/src/main/scala/kafka/controller/ControllerChannelManager.scala b/core/src/main/scala/kafka/controller/ControllerChannelManager.scala index addd88df3f06f..6121da420ad49 100755 --- a/core/src/main/scala/kafka/controller/ControllerChannelManager.scala +++ b/core/src/main/scala/kafka/controller/ControllerChannelManager.scala @@ -22,7 +22,6 @@ import java.util.concurrent.{BlockingQueue, LinkedBlockingQueue, TimeUnit} import com.yammer.metrics.core.{Gauge, Timer} import kafka.api._ import kafka.cluster.Broker -import kafka.common.KafkaException import kafka.metrics.KafkaMetricsGroup import kafka.server.KafkaConfig import kafka.utils._ @@ -35,7 +34,7 @@ import org.apache.kafka.common.requests._ import org.apache.kafka.common.security.JaasContext import org.apache.kafka.common.security.auth.SecurityProtocol import org.apache.kafka.common.utils.{LogContext, Time} -import org.apache.kafka.common.{Node, TopicPartition} +import org.apache.kafka.common.{KafkaException, Node, TopicPartition} import scala.collection.JavaConverters._ import scala.collection.mutable.HashMap @@ -292,6 +291,13 @@ class RequestSendThread(val controllerId: Int, } } + override def initiateShutdown(): Boolean = { + if (super.initiateShutdown()) { + networkClient.initiateClose() + true + } else + false + } } class ControllerBrokerRequestBatch(controller: KafkaController, stateChangeLogger: StateChangeLogger) extends Logging { diff --git a/core/src/main/scala/kafka/controller/KafkaController.scala b/core/src/main/scala/kafka/controller/KafkaController.scala index 9c33874e1cb7a..b8b44b8609b76 100644 --- a/core/src/main/scala/kafka/controller/KafkaController.scala +++ b/core/src/main/scala/kafka/controller/KafkaController.scala @@ -28,7 +28,7 @@ import kafka.utils._ import kafka.zk.KafkaZkClient.UpdateLeaderAndIsrResult import kafka.zk._ import kafka.zookeeper.{StateChangeHandler, ZNodeChangeHandler, ZNodeChildChangeHandler} -import org.apache.kafka.common.TopicPartition +import org.apache.kafka.common.{KafkaException, TopicPartition} import org.apache.kafka.common.errors.{BrokerNotAvailableException, ControllerMovedException} import org.apache.kafka.common.metrics.Metrics import org.apache.kafka.common.protocol.{ApiKeys, Errors} @@ -404,7 +404,8 @@ class KafkaController(val config: KafkaConfig, zkClient: KafkaZkClient, time: Ti deadBrokers.foreach(controllerContext.replicasOnOfflineDirs.remove) val deadBrokersThatWereShuttingDown = deadBrokers.filter(id => controllerContext.shuttingDownBrokerIds.remove(id)) - info(s"Removed $deadBrokersThatWereShuttingDown from list of shutting down brokers.") + if (deadBrokersThatWereShuttingDown.nonEmpty) + info(s"Removed ${deadBrokersThatWereShuttingDown.mkString(",")} from list of shutting down brokers.") val allReplicasOnDeadBrokers = controllerContext.replicasOnBrokers(deadBrokers.toSet) onReplicasBecomeOffline(allReplicasOnDeadBrokers) @@ -990,16 +991,13 @@ class KafkaController(val config: KafkaConfig, zkClient: KafkaZkClient, time: Ti // check ratio and if greater than desired ratio, trigger a rebalance for the topic partitions // that need to be on this broker if (imbalanceRatio > (config.leaderImbalancePerBrokerPercentage.toDouble / 100)) { - topicsNotInPreferredReplica.keys.foreach { topicPartition => - // do this check only if the broker is live and there are no partitions being reassigned currently - // and preferred replica election is not in progress - if (controllerContext.isReplicaOnline(leaderBroker, topicPartition) && - controllerContext.partitionsBeingReassigned.isEmpty && - !topicDeletionManager.isTopicQueuedUpForDeletion(topicPartition.topic) && - controllerContext.allTopics.contains(topicPartition.topic)) { - onPreferredReplicaElection(Set(topicPartition), isTriggeredByAutoRebalance = true) - } - } + // do this check only if the broker is live and there are no partitions being reassigned currently + // and preferred replica election is not in progress + val candidatePartitions = topicsNotInPreferredReplica.keys.filter(tp => controllerContext.isReplicaOnline(leaderBroker, tp) && + controllerContext.partitionsBeingReassigned.isEmpty && + !topicDeletionManager.isTopicQueuedUpForDeletion(tp.topic) && + controllerContext.allTopics.contains(tp.topic)) + onPreferredReplicaElection(candidatePartitions.toSet, isTriggeredByAutoRebalance = true) } } } diff --git a/core/src/main/scala/kafka/coordinator/group/GroupCoordinator.scala b/core/src/main/scala/kafka/coordinator/group/GroupCoordinator.scala index 9748e174c7847..2c062e2747bc6 100644 --- a/core/src/main/scala/kafka/coordinator/group/GroupCoordinator.scala +++ b/core/src/main/scala/kafka/coordinator/group/GroupCoordinator.scala @@ -779,6 +779,7 @@ class GroupCoordinator(val brokerId: Int, group.inLock { // remove any members who haven't joined the group yet group.notYetRejoinedMembers.foreach { failedMember => + removeHeartbeatForLeavingMember(group, failedMember) group.remove(failedMember.memberId) // TODO: cut the socket connection to the client } diff --git a/core/src/main/scala/kafka/coordinator/group/GroupMetadataManager.scala b/core/src/main/scala/kafka/coordinator/group/GroupMetadataManager.scala index 35a057430f047..bc2659ed299be 100644 --- a/core/src/main/scala/kafka/coordinator/group/GroupMetadataManager.scala +++ b/core/src/main/scala/kafka/coordinator/group/GroupMetadataManager.scala @@ -26,14 +26,14 @@ import java.util.concurrent.locks.ReentrantLock import com.yammer.metrics.core.Gauge import kafka.api.{ApiVersion, KAFKA_0_10_1_IV0} -import kafka.common.{KafkaException, MessageFormatter, OffsetAndMetadata} +import kafka.common.{MessageFormatter, OffsetAndMetadata} import kafka.metrics.KafkaMetricsGroup import kafka.server.ReplicaManager import kafka.utils.CoreUtils.inLock import kafka.utils._ import kafka.zk.KafkaZkClient import org.apache.kafka.clients.consumer.ConsumerRecord -import org.apache.kafka.common.TopicPartition +import org.apache.kafka.common.{KafkaException, TopicPartition} import org.apache.kafka.common.internals.Topic import org.apache.kafka.common.protocol.Errors import org.apache.kafka.common.protocol.types.Type._ @@ -517,7 +517,9 @@ class GroupMetadataManager(brokerId: Int, case Some(log) => var currOffset = log.logStartOffset - lazy val buffer = ByteBuffer.allocate(config.loadBufferSize) + + // buffer may not be needed if records are read from memory + var buffer = ByteBuffer.allocate(0) // loop breaks if leader changes at any time during the load, since getHighWatermark is -1 val loadedOffsets = mutable.Map[GroupTopicPartition, CommitRecordMetadataAndOffset]() @@ -531,25 +533,41 @@ class GroupMetadataManager(brokerId: Int, val memRecords = fetchDataInfo.records match { case records: MemoryRecords => records case fileRecords: FileRecords => - buffer.clear() - val bufferRead = fileRecords.readInto(buffer, 0) - MemoryRecords.readableRecords(bufferRead) + val sizeInBytes = fileRecords.sizeInBytes + val bytesNeeded = Math.max(config.loadBufferSize, sizeInBytes) + + // minOneMessage = true in the above log.read means that the buffer may need to be grown to ensure progress can be made + if (buffer.capacity < bytesNeeded) { + if (config.loadBufferSize < bytesNeeded) + warn(s"Loaded offsets and group metadata from $topicPartition with buffer larger ($bytesNeeded bytes) than " + + s"configured offsets.load.buffer.size (${config.loadBufferSize} bytes)") + + buffer = ByteBuffer.allocate(bytesNeeded) + } else { + buffer.clear() + } + + fileRecords.readInto(buffer, 0) + MemoryRecords.readableRecords(buffer) } memRecords.batches.asScala.foreach { batch => val isTxnOffsetCommit = batch.isTransactional if (batch.isControlBatch) { - val record = batch.iterator.next() - val controlRecord = ControlRecordType.parse(record.key) - if (controlRecord == ControlRecordType.COMMIT) { - pendingOffsets.getOrElse(batch.producerId, mutable.Map[GroupTopicPartition, CommitRecordMetadataAndOffset]()) - .foreach { - case (groupTopicPartition, commitRecordMetadataAndOffset) => - if (!loadedOffsets.contains(groupTopicPartition) || loadedOffsets(groupTopicPartition).olderThan(commitRecordMetadataAndOffset)) - loadedOffsets.put(groupTopicPartition, commitRecordMetadataAndOffset) - } + val recordIterator = batch.iterator + if (recordIterator.hasNext) { + val record = recordIterator.next() + val controlRecord = ControlRecordType.parse(record.key) + if (controlRecord == ControlRecordType.COMMIT) { + pendingOffsets.getOrElse(batch.producerId, mutable.Map[GroupTopicPartition, CommitRecordMetadataAndOffset]()) + .foreach { + case (groupTopicPartition, commitRecordMetadataAndOffset) => + if (!loadedOffsets.contains(groupTopicPartition) || loadedOffsets(groupTopicPartition).olderThan(commitRecordMetadataAndOffset)) + loadedOffsets.put(groupTopicPartition, commitRecordMetadataAndOffset) + } + } + pendingOffsets.remove(batch.producerId) } - pendingOffsets.remove(batch.producerId) } else { var batchBaseOffset: Option[Long] = None for (record <- batch.asScala) { diff --git a/core/src/main/scala/kafka/coordinator/transaction/ProducerIdManager.scala b/core/src/main/scala/kafka/coordinator/transaction/ProducerIdManager.scala index c3c9f7cf75c11..5c22c8e2cb23a 100644 --- a/core/src/main/scala/kafka/coordinator/transaction/ProducerIdManager.scala +++ b/core/src/main/scala/kafka/coordinator/transaction/ProducerIdManager.scala @@ -18,9 +18,9 @@ package kafka.coordinator.transaction import java.nio.charset.StandardCharsets -import kafka.common.KafkaException import kafka.utils.{Json, Logging} import kafka.zk.{KafkaZkClient, ProducerIdBlockZNode} +import org.apache.kafka.common.KafkaException import scala.collection.JavaConverters._ diff --git a/core/src/main/scala/kafka/coordinator/transaction/TransactionLog.scala b/core/src/main/scala/kafka/coordinator/transaction/TransactionLog.scala index 2c7178e885ecf..2dc6e38b44373 100644 --- a/core/src/main/scala/kafka/coordinator/transaction/TransactionLog.scala +++ b/core/src/main/scala/kafka/coordinator/transaction/TransactionLog.scala @@ -16,9 +16,9 @@ */ package kafka.coordinator.transaction -import kafka.common.{KafkaException, MessageFormatter} +import kafka.common.MessageFormatter import org.apache.kafka.clients.consumer.ConsumerRecord -import org.apache.kafka.common.TopicPartition +import org.apache.kafka.common.{KafkaException, TopicPartition} import org.apache.kafka.common.protocol.types.Type._ import org.apache.kafka.common.protocol.types._ import java.io.PrintStream diff --git a/core/src/main/scala/kafka/coordinator/transaction/TransactionMarkerChannelManager.scala b/core/src/main/scala/kafka/coordinator/transaction/TransactionMarkerChannelManager.scala index 7059ced5b3c02..f8b56e8e00800 100644 --- a/core/src/main/scala/kafka/coordinator/transaction/TransactionMarkerChannelManager.scala +++ b/core/src/main/scala/kafka/coordinator/transaction/TransactionMarkerChannelManager.scala @@ -135,7 +135,7 @@ class TransactionMarkerChannelManager(config: KafkaConfig, private val txnLogAppendRetryQueue = new LinkedBlockingQueue[TxnLogAppend]() - override val unsentExpiryMs: Int = config.requestTimeoutMs + override val requestTimeoutMs: Int = config.requestTimeoutMs newGauge( "UnknownDestinationQueueSize", diff --git a/core/src/main/scala/kafka/coordinator/transaction/TransactionStateManager.scala b/core/src/main/scala/kafka/coordinator/transaction/TransactionStateManager.scala index 5b82be4401bf3..2a4abb47a0de6 100644 --- a/core/src/main/scala/kafka/coordinator/transaction/TransactionStateManager.scala +++ b/core/src/main/scala/kafka/coordinator/transaction/TransactionStateManager.scala @@ -22,7 +22,6 @@ import java.util.concurrent.TimeUnit import java.util.concurrent.atomic.AtomicBoolean import java.util.concurrent.locks.ReentrantReadWriteLock -import kafka.common.KafkaException import kafka.log.LogConfig import kafka.message.UncompressedCodec import kafka.server.Defaults @@ -30,7 +29,7 @@ import kafka.server.ReplicaManager import kafka.utils.CoreUtils.{inReadLock, inWriteLock} import kafka.utils.{Logging, Pool, Scheduler} import kafka.zk.KafkaZkClient -import org.apache.kafka.common.TopicPartition +import org.apache.kafka.common.{KafkaException, TopicPartition} import org.apache.kafka.common.internals.Topic import org.apache.kafka.common.protocol.Errors import org.apache.kafka.common.record.{FileRecords, MemoryRecords, SimpleRecord} @@ -167,37 +166,32 @@ class TransactionStateManager(brokerId: Int, (topicPartition, records) } - def removeFromCacheCallback(responses: collection.Map[TopicPartition, PartitionResponse]): Unit = { responses.foreach { case (topicPartition, response) => - response.error match { - case Errors.NONE => - inReadLock(stateLock) { - val toRemove = transactionalIdByPartition(topicPartition.partition()) - transactionMetadataCache.get(topicPartition.partition) - .foreach { txnMetadataCacheEntry => - toRemove.foreach { idCoordinatorEpochAndMetadata => - val txnMetadata = txnMetadataCacheEntry.metadataPerTransactionalId.get(idCoordinatorEpochAndMetadata.transactionalId) - txnMetadata.inLock { - if (txnMetadataCacheEntry.coordinatorEpoch == idCoordinatorEpochAndMetadata.coordinatorEpoch - && txnMetadata.pendingState.contains(Dead) - && txnMetadata.producerEpoch == idCoordinatorEpochAndMetadata.transitMetadata.producerEpoch - ) - txnMetadataCacheEntry.metadataPerTransactionalId.remove(idCoordinatorEpochAndMetadata.transactionalId) - else { - debug(s"failed to remove expired transactionalId: ${idCoordinatorEpochAndMetadata.transactionalId}" + - s" from cache. pendingState: ${txnMetadata.pendingState} producerEpoch: ${txnMetadata.producerEpoch}" + - s" expected producerEpoch: ${idCoordinatorEpochAndMetadata.transitMetadata.producerEpoch}" + - s" coordinatorEpoch: ${txnMetadataCacheEntry.coordinatorEpoch} expected coordinatorEpoch: " + - s"${idCoordinatorEpochAndMetadata.coordinatorEpoch}") - txnMetadata.pendingState = None - } - } - } + inReadLock(stateLock) { + val toRemove = transactionalIdByPartition(topicPartition.partition) + transactionMetadataCache.get(topicPartition.partition).foreach { txnMetadataCacheEntry => + toRemove.foreach { idCoordinatorEpochAndMetadata => + val transactionalId = idCoordinatorEpochAndMetadata.transactionalId + val txnMetadata = txnMetadataCacheEntry.metadataPerTransactionalId.get(transactionalId) + txnMetadata.inLock { + if (txnMetadataCacheEntry.coordinatorEpoch == idCoordinatorEpochAndMetadata.coordinatorEpoch + && txnMetadata.pendingState.contains(Dead) + && txnMetadata.producerEpoch == idCoordinatorEpochAndMetadata.transitMetadata.producerEpoch + && response.error == Errors.NONE) { + txnMetadataCacheEntry.metadataPerTransactionalId.remove(transactionalId) + } else { + warn(s"Failed to remove expired transactionalId: $transactionalId" + + s" from cache. Tombstone append error code: ${response.error}," + + s" pendingState: ${txnMetadata.pendingState}, producerEpoch: ${txnMetadata.producerEpoch}," + + s" expected producerEpoch: ${idCoordinatorEpochAndMetadata.transitMetadata.producerEpoch}," + + s" coordinatorEpoch: ${txnMetadataCacheEntry.coordinatorEpoch}, expected coordinatorEpoch: " + + s"${idCoordinatorEpochAndMetadata.coordinatorEpoch}") + txnMetadata.pendingState = None } + } } - case _ => - debug(s"writing transactionalId tombstones for partition: ${topicPartition.partition} failed with error: ${response.error.message()}") + } } } } @@ -297,7 +291,8 @@ class TransactionStateManager(brokerId: Int, warn(s"Attempted to load offsets and group metadata from $topicPartition, but found no log") case Some(log) => - lazy val buffer = ByteBuffer.allocate(config.transactionLogLoadBufferSize) + // buffer may not be needed if records are read from memory + var buffer = ByteBuffer.allocate(0) // loop breaks if leader changes at any time during the load, since getHighWatermark is -1 var currOffset = log.logStartOffset @@ -312,9 +307,22 @@ class TransactionStateManager(brokerId: Int, val memRecords = fetchDataInfo.records match { case records: MemoryRecords => records case fileRecords: FileRecords => + val sizeInBytes = fileRecords.sizeInBytes + val bytesNeeded = Math.max(config.transactionLogLoadBufferSize, sizeInBytes) + + // minOneMessage = true in the above log.read means that the buffer may need to be grown to ensure progress can be made + if (buffer.capacity < bytesNeeded) { + if (config.transactionLogLoadBufferSize < bytesNeeded) + warn(s"Loaded offsets and group metadata from $topicPartition with buffer larger ($bytesNeeded bytes) than " + + s"configured transaction.state.log.load.buffer.size (${config.transactionLogLoadBufferSize} bytes)") + + buffer = ByteBuffer.allocate(bytesNeeded) + } else { + buffer.clear() + } buffer.clear() - val bufferRead = fileRecords.readInto(buffer, 0) - MemoryRecords.readableRecords(bufferRead) + fileRecords.readInto(buffer, 0) + MemoryRecords.readableRecords(buffer) } memRecords.batches.asScala.foreach { batch => diff --git a/core/src/main/scala/kafka/javaapi/FetchRequest.scala b/core/src/main/scala/kafka/javaapi/FetchRequest.scala deleted file mode 100644 index fe8beaac4dc49..0000000000000 --- a/core/src/main/scala/kafka/javaapi/FetchRequest.scala +++ /dev/null @@ -1,70 +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.javaapi - -import java.util - -import kafka.common.TopicAndPartition -import kafka.api.{PartitionFetchInfo, Request} - -import scala.collection.JavaConverters._ - -object FetchRequest { - private def seqToLinkedHashMap[K, V](s: Seq[(K, V)]): util.LinkedHashMap[K, V] = { - val map = new util.LinkedHashMap[K, V] - s.foreach { case (k, v) => map.put(k, v) } - map - } -} - -class FetchRequest(correlationId: Int, - clientId: String, - maxWait: Int, - minBytes: Int, - requestInfo: util.LinkedHashMap[TopicAndPartition, PartitionFetchInfo]) { - - @deprecated("The order of partitions in `requestInfo` is relevant, so this constructor is deprecated in favour of the " + - "one that takes a LinkedHashMap", since = "0.10.1.0") - def this(correlationId: Int, clientId: String, maxWait: Int, minBytes: Int, - requestInfo: java.util.Map[TopicAndPartition, PartitionFetchInfo]) { - this(correlationId, clientId, maxWait, minBytes, - FetchRequest.seqToLinkedHashMap(kafka.api.FetchRequest.shuffle(requestInfo.asScala.toSeq))) - } - - val underlying = kafka.api.FetchRequest( - correlationId = correlationId, - clientId = clientId, - replicaId = Request.OrdinaryConsumerId, - maxWait = maxWait, - minBytes = minBytes, - requestInfo = requestInfo.asScala.toBuffer - ) - - override def toString = underlying.toString - - override def equals(obj: Any): Boolean = { - obj match { - case null => false - case other: FetchRequest => this.underlying.equals(other.underlying) - case _ => false - } - } - - override def hashCode = underlying.hashCode -} - diff --git a/core/src/main/scala/kafka/javaapi/FetchResponse.scala b/core/src/main/scala/kafka/javaapi/FetchResponse.scala deleted file mode 100644 index c9165558014f8..0000000000000 --- a/core/src/main/scala/kafka/javaapi/FetchResponse.scala +++ /dev/null @@ -1,44 +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.javaapi - -class FetchResponse(private val underlying: kafka.api.FetchResponse) { - - def messageSet(topic: String, partition: Int): kafka.javaapi.message.ByteBufferMessageSet = { - import Implicits._ - underlying.messageSet(topic, partition) - } - - def highWatermark(topic: String, partition: Int) = underlying.highWatermark(topic, partition) - - def hasError = underlying.hasError - - def error(topic: String, partition: Int) = underlying.error(topic, partition) - - def errorCode(topic: String, partition: Int) = error(topic, partition).code - - override def equals(obj: Any): Boolean = { - obj match { - case null => false - case other: FetchResponse => this.underlying.equals(other.underlying) - case _ => false - } - } - - override def hashCode = underlying.hashCode -} diff --git a/core/src/main/scala/kafka/javaapi/GroupCoordinatorResponse.scala b/core/src/main/scala/kafka/javaapi/GroupCoordinatorResponse.scala deleted file mode 100644 index 096941c0c07a8..0000000000000 --- a/core/src/main/scala/kafka/javaapi/GroupCoordinatorResponse.scala +++ /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 kafka.javaapi - -import java.nio.ByteBuffer -import kafka.cluster.BrokerEndPoint - -class GroupCoordinatorResponse(private val underlying: kafka.api.GroupCoordinatorResponse) { - - def error = underlying.error - - def errorCode = error.code - - def coordinator: BrokerEndPoint = { - import kafka.javaapi.Implicits._ - underlying.coordinatorOpt - } - - override def equals(obj: Any): Boolean = { - obj match { - case null => false - case other: GroupCoordinatorResponse => this.underlying.equals(other.underlying) - case _ => false - } - } - - override def hashCode = underlying.hashCode - - override def toString = underlying.toString - -} - -object GroupCoordinatorResponse { - def readFrom(buffer: ByteBuffer) = new GroupCoordinatorResponse(kafka.api.GroupCoordinatorResponse.readFrom(buffer)) -} diff --git a/core/src/main/scala/kafka/javaapi/Implicits.scala b/core/src/main/scala/kafka/javaapi/Implicits.scala deleted file mode 100644 index c69b0a3c3d87e..0000000000000 --- a/core/src/main/scala/kafka/javaapi/Implicits.scala +++ /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 kafka.javaapi - -import kafka.utils.Logging - -private[javaapi] object Implicits extends Logging { - - implicit def scalaMessageSetToJavaMessageSet(messageSet: kafka.message.ByteBufferMessageSet): - kafka.javaapi.message.ByteBufferMessageSet = { - new kafka.javaapi.message.ByteBufferMessageSet(messageSet.buffer) - } - - implicit def toJavaFetchResponse(response: kafka.api.FetchResponse): kafka.javaapi.FetchResponse = - new kafka.javaapi.FetchResponse(response) - - implicit def toJavaTopicMetadataResponse(response: kafka.api.TopicMetadataResponse): kafka.javaapi.TopicMetadataResponse = - new kafka.javaapi.TopicMetadataResponse(response) - - implicit def toJavaOffsetResponse(response: kafka.api.OffsetResponse): kafka.javaapi.OffsetResponse = - new kafka.javaapi.OffsetResponse(response) - - implicit def toJavaOffsetFetchResponse(response: kafka.api.OffsetFetchResponse): kafka.javaapi.OffsetFetchResponse = - new kafka.javaapi.OffsetFetchResponse(response) - - implicit def toJavaOffsetCommitResponse(response: kafka.api.OffsetCommitResponse): kafka.javaapi.OffsetCommitResponse = - new kafka.javaapi.OffsetCommitResponse(response) - - implicit def optionToJavaRef[T](opt: Option[T]): T = { - opt match { - case Some(obj) => obj - case None => null.asInstanceOf[T] - } - } - -} diff --git a/core/src/main/scala/kafka/javaapi/OffsetCommitRequest.scala b/core/src/main/scala/kafka/javaapi/OffsetCommitRequest.scala deleted file mode 100644 index 0c3c6516b07ae..0000000000000 --- a/core/src/main/scala/kafka/javaapi/OffsetCommitRequest.scala +++ /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 kafka.javaapi - -import kafka.common.{OffsetAndMetadata, TopicAndPartition} -import scala.collection.JavaConverters._ - -class OffsetCommitRequest(groupId: String, - requestInfo: java.util.Map[TopicAndPartition, OffsetAndMetadata], - correlationId: Int, - clientId: String, - versionId: Short) { - val underlying = { - val scalaMap: collection.immutable.Map[TopicAndPartition, OffsetAndMetadata] = requestInfo.asScala.toMap - 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 - - override def equals(obj: Any): Boolean = { - obj match { - case null => false - case other: OffsetCommitRequest => this.underlying.equals(other.underlying) - case _ => false - } - } - - override def hashCode = underlying.hashCode -} diff --git a/core/src/main/scala/kafka/javaapi/OffsetCommitResponse.scala b/core/src/main/scala/kafka/javaapi/OffsetCommitResponse.scala deleted file mode 100644 index c348eba22529e..0000000000000 --- a/core/src/main/scala/kafka/javaapi/OffsetCommitResponse.scala +++ /dev/null @@ -1,39 +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.javaapi - -import java.nio.ByteBuffer - -import kafka.common.TopicAndPartition -import org.apache.kafka.common.protocol.Errors -import scala.collection.JavaConverters._ - -class OffsetCommitResponse(private val underlying: kafka.api.OffsetCommitResponse) { - - def errors: java.util.Map[TopicAndPartition, Errors] = underlying.commitStatus.asJava - - def hasError = underlying.hasError - - def error(topicAndPartition: TopicAndPartition) = underlying.commitStatus(topicAndPartition) - - def errorCode(topicAndPartition: TopicAndPartition) = error(topicAndPartition).code -} - -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 deleted file mode 100644 index 5f96439fba534..0000000000000 --- a/core/src/main/scala/kafka/javaapi/OffsetFetchRequest.scala +++ /dev/null @@ -1,60 +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.javaapi - -import kafka.common.TopicAndPartition -import collection.JavaConverters._ - -class OffsetFetchRequest(groupId: String, - requestInfo: java.util.List[TopicAndPartition], - versionId: Short, - 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 = { - kafka.api.OffsetFetchRequest( - groupId = groupId, - requestInfo = requestInfo.asScala, - versionId = versionId, - correlationId = correlationId, - clientId = clientId - ) - } - - override def toString = underlying.toString - - override def equals(obj: Any): Boolean = { - obj match { - case null => false - case other: OffsetFetchRequest => this.underlying.equals(other.underlying) - case _ => false - } - } - - override def hashCode = underlying.hashCode -} - - diff --git a/core/src/main/scala/kafka/javaapi/OffsetFetchResponse.scala b/core/src/main/scala/kafka/javaapi/OffsetFetchResponse.scala deleted file mode 100644 index 01aa8e8e88f14..0000000000000 --- a/core/src/main/scala/kafka/javaapi/OffsetFetchResponse.scala +++ /dev/null @@ -1,33 +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.javaapi - -import java.nio.ByteBuffer - -import kafka.common.{TopicAndPartition, OffsetMetadataAndError} -import collection.JavaConverters._ - -class OffsetFetchResponse(private val underlying: kafka.api.OffsetFetchResponse) { - - def offsets: java.util.Map[TopicAndPartition, OffsetMetadataAndError] = underlying.requestInfo.asJava - -} - -object OffsetFetchResponse { - def readFrom(buffer: ByteBuffer) = new OffsetFetchResponse(kafka.api.OffsetFetchResponse.readFrom(buffer)) -} diff --git a/core/src/main/scala/kafka/javaapi/OffsetRequest.scala b/core/src/main/scala/kafka/javaapi/OffsetRequest.scala deleted file mode 100644 index 96b66ef6ee615..0000000000000 --- a/core/src/main/scala/kafka/javaapi/OffsetRequest.scala +++ /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 kafka.javaapi - -import kafka.common.TopicAndPartition -import kafka.api.{Request, PartitionOffsetRequestInfo} -import scala.collection.JavaConverters._ - - -class OffsetRequest(requestInfo: java.util.Map[TopicAndPartition, PartitionOffsetRequestInfo], - versionId: Short, - clientId: String) { - - val underlying = { - val scalaMap = requestInfo.asScala.toMap - kafka.api.OffsetRequest( - requestInfo = scalaMap, - versionId = versionId, - clientId = clientId, - replicaId = Request.OrdinaryConsumerId - ) - } - - override def toString = underlying.toString - - override def equals(obj: Any): Boolean = { - obj match { - case null => false - case other: OffsetRequest => this.underlying.equals(other.underlying) - case _ => false - } - } - - override def hashCode = underlying.hashCode -} diff --git a/core/src/main/scala/kafka/javaapi/OffsetResponse.scala b/core/src/main/scala/kafka/javaapi/OffsetResponse.scala deleted file mode 100644 index cb2047f43e17e..0000000000000 --- a/core/src/main/scala/kafka/javaapi/OffsetResponse.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.javaapi - -import kafka.common.TopicAndPartition - -class OffsetResponse(private val underlying: kafka.api.OffsetResponse) { - - def hasError = underlying.hasError - - def error(topic: String, partition: Int) = - underlying.partitionErrorAndOffsets(TopicAndPartition(topic, partition)).error - - def errorCode(topic: String, partition: Int) = error(topic, partition).code - - def offsets(topic: String, partition: Int) = - underlying.partitionErrorAndOffsets(TopicAndPartition(topic, partition)).offsets.toArray - - override def equals(obj: Any): Boolean = { - obj match { - case null => false - case other: OffsetResponse => this.underlying.equals(other.underlying) - case _ => false - } - } - - override def hashCode = underlying.hashCode - - override def toString = underlying.toString -} diff --git a/core/src/main/scala/kafka/javaapi/TopicMetadata.scala b/core/src/main/scala/kafka/javaapi/TopicMetadata.scala deleted file mode 100644 index 051445ceba547..0000000000000 --- a/core/src/main/scala/kafka/javaapi/TopicMetadata.scala +++ /dev/null @@ -1,67 +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.javaapi - -import kafka.cluster.BrokerEndPoint -import scala.collection.JavaConverters._ - -private[javaapi] object MetadataListImplicits { - implicit def toJavaTopicMetadataList(topicMetadataSeq: Seq[kafka.api.TopicMetadata]): - java.util.List[kafka.javaapi.TopicMetadata] = topicMetadataSeq.map(new kafka.javaapi.TopicMetadata(_)).asJava - - implicit def toPartitionMetadataList(partitionMetadataSeq: Seq[kafka.api.PartitionMetadata]): - java.util.List[kafka.javaapi.PartitionMetadata] = partitionMetadataSeq.map(new kafka.javaapi.PartitionMetadata(_)).asJava -} - -class TopicMetadata(private val underlying: kafka.api.TopicMetadata) { - def topic: String = underlying.topic - - def partitionsMetadata: java.util.List[PartitionMetadata] = { - import kafka.javaapi.MetadataListImplicits._ - underlying.partitionsMetadata - } - - def error = underlying.error - - def errorCode = error.code - - def sizeInBytes: Int = underlying.sizeInBytes - - override def toString = underlying.toString -} - - -class PartitionMetadata(private val underlying: kafka.api.PartitionMetadata) { - def partitionId: Int = underlying.partitionId - - def leader: BrokerEndPoint = { - import kafka.javaapi.Implicits._ - underlying.leader - } - - def replicas: java.util.List[BrokerEndPoint] = underlying.replicas.asJava - - def isr: java.util.List[BrokerEndPoint] = underlying.isr.asJava - - def error = underlying.error - - def errorCode = error.code - - def sizeInBytes: Int = underlying.sizeInBytes - - override def toString = underlying.toString -} diff --git a/core/src/main/scala/kafka/javaapi/TopicMetadataRequest.scala b/core/src/main/scala/kafka/javaapi/TopicMetadataRequest.scala deleted file mode 100644 index fdb14cbaeaff0..0000000000000 --- a/core/src/main/scala/kafka/javaapi/TopicMetadataRequest.scala +++ /dev/null @@ -1,66 +0,0 @@ -/** - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package kafka.javaapi - -import java.nio.ByteBuffer - -import kafka.api._ -import org.apache.kafka.common.protocol.ApiKeys - -import scala.collection.JavaConverters._ - -class TopicMetadataRequest(val versionId: Short, - val correlationId: Int, - val clientId: String, - val topics: java.util.List[String]) - extends RequestOrResponse(Some(ApiKeys.METADATA.id)) { - - val underlying: kafka.api.TopicMetadataRequest = new kafka.api.TopicMetadataRequest(versionId, correlationId, clientId, topics.asScala) - - def this(topics: java.util.List[String]) = - this(kafka.api.TopicMetadataRequest.CurrentVersion, 0, kafka.api.TopicMetadataRequest.DefaultClientId, topics) - - def this(topics: java.util.List[String], correlationId: Int) = - this(kafka.api.TopicMetadataRequest.CurrentVersion, correlationId, kafka.api.TopicMetadataRequest.DefaultClientId, topics) - - def writeTo(buffer: ByteBuffer) = underlying.writeTo(buffer) - - def sizeInBytes: Int = underlying.sizeInBytes - - override def toString: String = { - describe(true) - } - - override def describe(details: Boolean): String = { - val topicMetadataRequest = new StringBuilder - topicMetadataRequest.append("Name: " + this.getClass.getSimpleName) - topicMetadataRequest.append("; Version: " + versionId) - topicMetadataRequest.append("; CorrelationId: " + correlationId) - topicMetadataRequest.append("; ClientId: " + clientId) - if(details) { - topicMetadataRequest.append("; Topics: ") - val topicIterator = topics.iterator() - while (topicIterator.hasNext) { - val topic = topicIterator.next() - topicMetadataRequest.append("%s".format(topic)) - if(topicIterator.hasNext) - topicMetadataRequest.append(",") - } - } - topicMetadataRequest.toString() - } -} diff --git a/core/src/main/scala/kafka/javaapi/TopicMetadataResponse.scala b/core/src/main/scala/kafka/javaapi/TopicMetadataResponse.scala deleted file mode 100644 index 40f81d553611a..0000000000000 --- a/core/src/main/scala/kafka/javaapi/TopicMetadataResponse.scala +++ /dev/null @@ -1,38 +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.javaapi - -class TopicMetadataResponse(private val underlying: kafka.api.TopicMetadataResponse) { - def sizeInBytes: Int = underlying.sizeInBytes - - def topicsMetadata: java.util.List[kafka.javaapi.TopicMetadata] = { - import kafka.javaapi.MetadataListImplicits._ - underlying.topicsMetadata - } - - override def equals(obj: Any): Boolean = { - obj match { - case null => false - case other: TopicMetadataResponse => this.underlying.equals(other.underlying) - case _ => false - } - } - - override def hashCode = underlying.hashCode - - override def toString = underlying.toString -} diff --git a/core/src/main/scala/kafka/javaapi/consumer/ConsumerConnector.java b/core/src/main/scala/kafka/javaapi/consumer/ConsumerConnector.java deleted file mode 100644 index def977cfaf64a..0000000000000 --- a/core/src/main/scala/kafka/javaapi/consumer/ConsumerConnector.java +++ /dev/null @@ -1,91 +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.javaapi.consumer; - -import kafka.common.OffsetAndMetadata; -import kafka.common.TopicAndPartition; -import kafka.consumer.KafkaStream; -import kafka.consumer.TopicFilter; -import kafka.serializer.Decoder; - -import java.util.List; -import java.util.Map; - -/** - * @deprecated since 0.11.0.0, this interface will be removed in a future release. - */ -@Deprecated -public interface ConsumerConnector { - /** - * Create a list of MessageStreams of type T for each topic. - * - * @param topicCountMap a map of (topic, #streams) pair - * @param keyDecoder a decoder that decodes the message key - * @param valueDecoder a decoder that decodes the message itself - * @return a map of (topic, list of KafkaStream) pairs. - * The number of items in the list is #streams. Each stream supports - * an iterator over message/metadata pairs. - */ - public Map>> - createMessageStreams(Map topicCountMap, Decoder keyDecoder, Decoder valueDecoder); - - public Map>> createMessageStreams(Map topicCountMap); - - /** - * Create a list of MessageAndTopicStreams containing messages of type T. - * - * @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 List> - createMessageStreamsByFilter(TopicFilter topicFilter, int numStreams, Decoder keyDecoder, Decoder valueDecoder); - - public List> createMessageStreamsByFilter(TopicFilter topicFilter, int numStreams); - - public List> createMessageStreamsByFilter(TopicFilter topicFilter); - - /** - * Commit the offsets of all broker partitions connected by this connector. - */ - public void commitOffsets(); - - public void commitOffsets(boolean retryOnFailure); - - /** - * Commit offsets using the provided offsets map - * - * @param offsetsToCommit a map containing the offset to commit for each partition. - * @param retryOnFailure enable retries on the offset commit if it fails. - */ - public void commitOffsets(Map offsetsToCommit, boolean retryOnFailure); - - /** - * 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 - */ - public void shutdown(); -} diff --git a/core/src/main/scala/kafka/javaapi/consumer/ConsumerRebalanceListener.java b/core/src/main/scala/kafka/javaapi/consumer/ConsumerRebalanceListener.java deleted file mode 100644 index ff23760a04792..0000000000000 --- a/core/src/main/scala/kafka/javaapi/consumer/ConsumerRebalanceListener.java +++ /dev/null @@ -1,54 +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.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} - */ -/** - * @deprecated since 0.11.0.0, this interface will be removed in a future release. - */ -@Deprecated -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 deleted file mode 100644 index 188babbe8293b..0000000000000 --- a/core/src/main/scala/kafka/javaapi/consumer/SimpleConsumer.scala +++ /dev/null @@ -1,107 +0,0 @@ -/** - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. -*/ - -package kafka.javaapi.consumer - -import kafka.utils.threadsafe -import kafka.javaapi.FetchResponse -import kafka.javaapi.OffsetRequest - -/** - * A consumer of kafka messages - */ -@deprecated("This class has been deprecated and will be removed in a future release. " + - "Please use org.apache.kafka.clients.consumer.KafkaConsumer instead.", "0.11.0.0") -@threadsafe -class SimpleConsumer(val host: String, - val port: Int, - val soTimeout: Int, - val bufferSize: Int, - val clientId: String) { - - private val underlying = new kafka.consumer.SimpleConsumer(host, port, soTimeout, bufferSize, clientId) - - /** - * Fetch a set of messages from a topic. This version of the fetch method - * takes the Scala version of a fetch request (i.e., - * [[kafka.api.FetchRequest]] and is intended for use with the - * [[kafka.api.FetchRequestBuilder]]. - * - * @param request specifies the topic name, topic partition, starting byte offset, maximum bytes to be fetched. - * @return a set of fetched messages - */ - def fetch(request: kafka.api.FetchRequest): FetchResponse = { - import kafka.javaapi.Implicits._ - underlying.fetch(request) - } - - /** - * 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 - */ - def fetch(request: kafka.javaapi.FetchRequest): FetchResponse = { - fetch(request.underlying) - } - - /** - * Fetch metadata for a sequence of topics. - * - * @param request specifies the versionId, clientId, sequence of topics. - * @return metadata for each topic in the request. - */ - def send(request: kafka.javaapi.TopicMetadataRequest): kafka.javaapi.TopicMetadataResponse = { - import kafka.javaapi.Implicits._ - underlying.send(request.underlying) - } - - /** - * 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. - */ - def getOffsetsBefore(request: OffsetRequest): kafka.javaapi.OffsetResponse = { - import kafka.javaapi.Implicits._ - underlying.getOffsetsBefore(request.underlying) - } - - /** - * Commit offsets for a topic to Zookeeper - * @param request a [[kafka.javaapi.OffsetCommitRequest]] object. - * @return a [[kafka.javaapi.OffsetCommitResponse]] object. - */ - def commitOffsets(request: kafka.javaapi.OffsetCommitRequest): kafka.javaapi.OffsetCommitResponse = { - import kafka.javaapi.Implicits._ - underlying.commitOffsets(request.underlying) - } - - /** - * Fetch offsets for a topic from Zookeeper - * @param request a [[kafka.javaapi.OffsetFetchRequest]] object. - * @return a [[kafka.javaapi.OffsetFetchResponse]] object. - */ - def fetchOffsets(request: kafka.javaapi.OffsetFetchRequest): kafka.javaapi.OffsetFetchResponse = { - import kafka.javaapi.Implicits._ - underlying.fetchOffsets(request.underlying) - } - - def close() { - underlying.close - } -} diff --git a/core/src/main/scala/kafka/javaapi/consumer/ZookeeperConsumerConnector.scala b/core/src/main/scala/kafka/javaapi/consumer/ZookeeperConsumerConnector.scala deleted file mode 100644 index d646938392d36..0000000000000 --- a/core/src/main/scala/kafka/javaapi/consumer/ZookeeperConsumerConnector.scala +++ /dev/null @@ -1,126 +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.javaapi.consumer - -import kafka.serializer._ -import kafka.consumer._ -import kafka.common.{OffsetAndMetadata, TopicAndPartition, MessageStreamsExistException} -import java.util.concurrent.atomic.AtomicBoolean -import scala.collection.JavaConverters._ - -/** - * This class handles the consumers interaction with zookeeper - * - * Directories: - * 1. Consumer id registry: - * /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. - * - * The consumer id is picked up from configuration, instead of the sequential id assigned by ZK. Generated sequential - * ids are hard to recover during temporary connection loss to ZK, since it's difficult for the client to figure out - * whether the creation of a sequential znode has succeeded or not. More details can be found at - * (http://wiki.apache.org/hadoop/ZooKeeper/ErrorHandling) - * - * 2. Broker node registry: - * /brokers/[0...N] --> { "host" : "host:port", - * "topics" : {"topic1": ["partition1" ... "partitionN"], ..., - * "topicN": ["partition1" ... "partitionN"] } } - * This is a list of all present broker brokers. A unique logical node id is configured on each broker node. A broker - * node registers itself on start-up and creates a znode with the logical node id under /brokers. The value of the znode - * is a JSON String that contains (1) the host name and the port the broker is listening to, (2) a list of topics that - * the broker serves, (3) a list of logical partitions assigned to each topic on the broker. - * A consumer subscribes to event changes of the broker node registry. - * - * 3. Partition owner registry: - * /consumers/[group_id]/owner/[topic]/[broker_id-partition_id] --> consumer_node_id - * This stores the mapping before broker partitions and consumers. Each partition is owned by a unique consumer - * within a consumer group. The mapping is reestablished after each rebalancing. - * - * 4. Consumer offset tracking: - * /consumers/[group_id]/offsets/[topic]/[broker_id-partition_id] --> offset_counter_value - * Each consumer tracks the offset of the latest message consumed for each partition. - * -*/ - -@deprecated("This class has been deprecated and will be removed in a future release.", "0.11.0.0") -private[kafka] class ZookeeperConsumerConnector(val config: ConsumerConfig, - val enableFetcher: Boolean) // for testing only - extends ConsumerConnector { - - private val underlying = new kafka.consumer.ZookeeperConsumerConnector(config, enableFetcher) - private val messageStreamCreated = new AtomicBoolean(false) - - def this(config: ConsumerConfig) = this(config, true) - - // for java client - def createMessageStreams[K,V]( - topicCountMap: java.util.Map[String,java.lang.Integer], - keyDecoder: Decoder[K], - valueDecoder: Decoder[V]) - : java.util.Map[String,java.util.List[KafkaStream[K,V]]] = { - - if (messageStreamCreated.getAndSet(true)) - throw new MessageStreamsExistException(this.getClass.getSimpleName + - " can create message streams at most once",null) - val scalaTopicCountMap: Map[String, Int] = { - Map.empty[String, Int] ++ topicCountMap.asInstanceOf[java.util.Map[String, Int]].asScala - } - val scalaReturn = underlying.consume(scalaTopicCountMap, keyDecoder, valueDecoder) - val ret = new java.util.HashMap[String,java.util.List[KafkaStream[K,V]]] - for ((topic, streams) <- scalaReturn) { - val javaStreamList = new java.util.ArrayList[KafkaStream[K,V]] - for (stream <- streams) - javaStreamList.add(stream) - ret.put(topic, javaStreamList) - } - ret - } - - def createMessageStreams(topicCountMap: java.util.Map[String,java.lang.Integer]): java.util.Map[String,java.util.List[KafkaStream[Array[Byte],Array[Byte]]]] = - createMessageStreams(topicCountMap, new DefaultDecoder(), new DefaultDecoder()) - - def createMessageStreamsByFilter[K,V](topicFilter: TopicFilter, numStreams: Int, keyDecoder: Decoder[K], valueDecoder: Decoder[V]) = - underlying.createMessageStreamsByFilter(topicFilter, numStreams, keyDecoder, valueDecoder).asJava - - def createMessageStreamsByFilter(topicFilter: TopicFilter, numStreams: Int) = - createMessageStreamsByFilter(topicFilter, numStreams, new DefaultDecoder(), new DefaultDecoder()) - - def createMessageStreamsByFilter(topicFilter: TopicFilter) = - createMessageStreamsByFilter(topicFilter, 1, new DefaultDecoder(), new DefaultDecoder()) - - def commitOffsets() { - underlying.commitOffsets(true) - } - - def commitOffsets(retryOnFailure: Boolean) { - underlying.commitOffsets(retryOnFailure) - } - - def commitOffsets(offsetsToCommit: java.util.Map[TopicAndPartition, OffsetAndMetadata], retryOnFailure: Boolean) { - underlying.commitOffsets(offsetsToCommit.asScala.toMap, 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 deleted file mode 100644 index 590db83be42df..0000000000000 --- a/core/src/main/scala/kafka/javaapi/message/ByteBufferMessageSet.scala +++ /dev/null @@ -1,67 +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.javaapi.message - -import java.nio.ByteBuffer - -import kafka.common.LongRef -import kafka.message._ - -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]) { - this(new kafka.message.ByteBufferMessageSet(compressionCodec, new LongRef(0), messages.asScala: _*).buffer) - } - - def this(messages: java.util.List[Message]) { - this(NoCompressionCodec, messages) - } - - def validBytes: Int = underlying.validBytes - - def getBuffer = buffer - - override def iterator: java.util.Iterator[MessageAndOffset] = new java.util.Iterator[MessageAndOffset] { - val underlyingIterator = underlying.iterator - override def hasNext(): Boolean = { - underlyingIterator.hasNext - } - - override def next(): MessageAndOffset = { - underlyingIterator.next - } - - override def remove = throw new UnsupportedOperationException("remove API on MessageSet is not supported") - } - - override def toString: String = underlying.toString - - def sizeInBytes: Int = underlying.sizeInBytes - - override def equals(other: Any): Boolean = { - other match { - case that: ByteBufferMessageSet => buffer.equals(that.buffer) - case _ => false - } - } - - - override def hashCode: Int = buffer.hashCode -} diff --git a/core/src/main/scala/kafka/javaapi/message/MessageSet.scala b/core/src/main/scala/kafka/javaapi/message/MessageSet.scala deleted file mode 100644 index 80a67cd3546be..0000000000000 --- a/core/src/main/scala/kafka/javaapi/message/MessageSet.scala +++ /dev/null @@ -1,55 +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.javaapi.message - - -import kafka.message.{MessageAndOffset, InvalidMessageException} - - -/** - * A set of messages. A message set has a fixed serialized form, though the container - * for the bytes could be either in-memory or on disk. A The format of each message is - * as follows: - * 4 byte size containing an integer N - * N message bytes as described in the message class - */ -abstract class MessageSet extends java.lang.Iterable[MessageAndOffset] { - - /** - * Provides an iterator over the messages in this set - */ - def iterator: java.util.Iterator[MessageAndOffset] - - /** - * Gives the total size of this message set in bytes - */ - def sizeInBytes: Int - - /** - * Validate the checksum of all the messages in the set. Throws an InvalidMessageException if the checksum doesn't - * match the payload for any message. - */ - def validate(): Unit = { - val thisIterator = this.iterator - while(thisIterator.hasNext) { - val messageAndOffset = thisIterator.next - if(!messageAndOffset.message.isValid) - throw new InvalidMessageException - } - } -} diff --git a/core/src/main/scala/kafka/log/AbstractIndex.scala b/core/src/main/scala/kafka/log/AbstractIndex.scala index 44083c186c86a..2be776c61c84e 100644 --- a/core/src/main/scala/kafka/log/AbstractIndex.scala +++ b/core/src/main/scala/kafka/log/AbstractIndex.scala @@ -18,11 +18,12 @@ package kafka.log import java.io.{File, RandomAccessFile} -import java.nio.{ByteBuffer, MappedByteBuffer} import java.nio.channels.FileChannel import java.nio.file.Files +import java.nio.{ByteBuffer, MappedByteBuffer} import java.util.concurrent.locks.{Lock, ReentrantLock} +import kafka.common.IndexOffsetOverflowException import kafka.log.IndexSearchType.IndexSearchEntity import kafka.utils.CoreUtils.inLock import kafka.utils.{CoreUtils, Logging} @@ -115,6 +116,7 @@ abstract class AbstractIndex[K, V](@volatile var file: File, val baseOffset: Lon val roundedNewSize = roundDownToExactMultiple(newSize, entrySize) if (_length == roundedNewSize) { + debug(s"Index ${file.getAbsolutePath} was not resized because it already has size $roundedNewSize") false } else { val raf = new RandomAccessFile(file, "rw") @@ -129,6 +131,8 @@ abstract class AbstractIndex[K, V](@volatile var file: File, val baseOffset: Lon mmap = raf.getChannel().map(FileChannel.MapMode.READ_WRITE, 0, roundedNewSize) _maxEntries = mmap.limit() / entrySize mmap.position(position) + debug(s"Resized ${file.getAbsolutePath} to $roundedNewSize, position is ${mmap.position()} " + + s"and limit is ${mmap.limit()}") true } finally { CoreUtils.swallow(raf.close(), this) @@ -226,6 +230,26 @@ abstract class AbstractIndex[K, V](@volatile var file: File, val baseOffset: Lon resize(maxIndexSize) } + /** + * Get offset relative to base offset of this index + * @throws IndexOffsetOverflowException + */ + def relativeOffset(offset: Long): Int = { + val relativeOffset = toRelative(offset) + if (relativeOffset.isEmpty) + throw new IndexOffsetOverflowException(s"Integer overflow for offset: $offset (${file.getAbsoluteFile})") + relativeOffset.get + } + + /** + * Check if a particular offset is valid to be appended to this index. + * @param offset The offset to check + * @return true if this offset is valid to be appended to this index; false otherwise + */ + def canAppendOffset(offset: Long): Boolean = { + toRelative(offset).isDefined + } + protected def safeForceUnmap(): Unit = { try forceUnmap() catch { @@ -325,6 +349,14 @@ abstract class AbstractIndex[K, V](@volatile var file: File, val baseOffset: Lon */ private def roundDownToExactMultiple(number: Int, factor: Int) = factor * (number / factor) + private def toRelative(offset: Long): Option[Int] = { + val relativeOffset = offset - baseOffset + if (relativeOffset < 0 || relativeOffset > Int.MaxValue) + None + else + Some(relativeOffset.toInt) + } + } object IndexSearchType extends Enumeration { diff --git a/core/src/main/scala/kafka/log/Log.scala b/core/src/main/scala/kafka/log/Log.scala index 777dbb56d891e..cd48bde51ce2a 100644 --- a/core/src/main/scala/kafka/log/Log.scala +++ b/core/src/main/scala/kafka/log/Log.scala @@ -18,41 +18,42 @@ package kafka.log import java.io.{File, IOException} +import java.lang.{Long => JLong} import java.nio.file.{Files, NoSuchFileException} import java.text.NumberFormat +import java.util.Map.{Entry => JEntry} import java.util.concurrent.atomic._ import java.util.concurrent.{ConcurrentNavigableMap, ConcurrentSkipListMap, TimeUnit} +import java.util.regex.Pattern +import com.yammer.metrics.core.Gauge import kafka.api.KAFKA_0_10_0_IV0 -import kafka.common.{InvalidOffsetException, KafkaException, LongRef} +import kafka.common.{LogSegmentOffsetOverflowException, LongRef, OffsetsOutOfOrderException, UnexpectedAppendOffsetException} +import kafka.message.{BrokerCompressionCodec, CompressionCodec, NoCompressionCodec} import kafka.metrics.KafkaMetricsGroup +import kafka.server.checkpoints.{LeaderEpochCheckpointFile, LeaderEpochFile} +import kafka.server.epoch.LeaderEpochFileCache import kafka.server.{BrokerTopicStats, FetchDataInfo, LogDirFailureChannel, LogOffsetMetadata} import kafka.utils._ -import org.apache.kafka.common.errors.{CorruptRecordException, KafkaStorageException, OffsetOutOfRangeException, RecordBatchTooLargeException, RecordTooLargeException, UnsupportedForMessageFormatException} +import org.apache.kafka.common.errors._ import org.apache.kafka.common.record._ +import org.apache.kafka.common.requests.EpochEndOffset.UNDEFINED_EPOCH +import org.apache.kafka.common.requests.FetchResponse.AbortedTransaction import org.apache.kafka.common.requests.{IsolationLevel, ListOffsetRequest} +import org.apache.kafka.common.utils.{Time, Utils} +import org.apache.kafka.common.{KafkaException, TopicPartition} import scala.collection.JavaConverters._ import scala.collection.mutable.{ArrayBuffer, ListBuffer} import scala.collection.{Seq, Set, mutable} -import com.yammer.metrics.core.Gauge -import org.apache.kafka.common.utils.{Time, Utils} -import kafka.message.{BrokerCompressionCodec, CompressionCodec, NoCompressionCodec} -import kafka.server.checkpoints.{LeaderEpochCheckpointFile, LeaderEpochFile} -import kafka.server.epoch.{LeaderEpochCache, LeaderEpochFileCache} -import org.apache.kafka.common.TopicPartition -import org.apache.kafka.common.requests.FetchResponse.AbortedTransaction -import java.util.Map.{Entry => JEntry} -import java.lang.{Long => JLong} -import java.util.regex.Pattern object LogAppendInfo { val UnknownLogAppendInfo = LogAppendInfo(None, -1, RecordBatch.NO_TIMESTAMP, -1L, RecordBatch.NO_TIMESTAMP, -1L, - RecordConversionStats.EMPTY, NoCompressionCodec, NoCompressionCodec, -1, -1, offsetsMonotonic = false) + RecordConversionStats.EMPTY, NoCompressionCodec, NoCompressionCodec, -1, -1, offsetsMonotonic = false, -1L) def unknownLogAppendInfoWithLogStartOffset(logStartOffset: Long): LogAppendInfo = LogAppendInfo(None, -1, RecordBatch.NO_TIMESTAMP, -1L, RecordBatch.NO_TIMESTAMP, logStartOffset, - RecordConversionStats.EMPTY, NoCompressionCodec, NoCompressionCodec, -1, -1, offsetsMonotonic = false) + RecordConversionStats.EMPTY, NoCompressionCodec, NoCompressionCodec, -1, -1, offsetsMonotonic = false, -1L) } /** @@ -71,6 +72,7 @@ object LogAppendInfo { * @param shallowCount The number of shallow messages * @param validBytes The number of valid bytes * @param offsetsMonotonic Are the offsets in this message set monotonically increasing + * @param lastOffsetOfFirstBatch The last offset of the first batch */ case class LogAppendInfo(var firstOffset: Option[Long], var lastOffset: Long, @@ -83,17 +85,20 @@ case class LogAppendInfo(var firstOffset: Option[Long], targetCodec: CompressionCodec, shallowCount: Int, validBytes: Int, - offsetsMonotonic: Boolean) { + offsetsMonotonic: Boolean, + lastOffsetOfFirstBatch: Long) { /** - * Get the first offset if it exists, else get the last offset. - * @return The offset of first message if it exists; else offset of the last message. - */ - def firstOrLastOffset: Long = firstOffset.getOrElse(lastOffset) + * Get the first offset if it exists, else get the last offset of the first batch + * For magic versions 2 and newer, this method will return first offset. For magic versions + * older than 2, we use the last offset of the first batch as an approximation of the first + * offset to avoid decompressing the data. + */ + def firstOrLastOffsetOfFirstBatch: Long = firstOffset.getOrElse(lastOffsetOfFirstBatch) /** - * Get the (maximum) number of messages described by LogAppendInfo - * @return Maximum possible number of messages described by LogAppendInfo - */ + * Get the (maximum) number of messages described by LogAppendInfo + * @return Maximum possible number of messages described by LogAppendInfo + */ def numMessages: Long = { firstOffset match { case Some(firstOffsetVal) if (firstOffsetVal >= 0 && lastOffset >= 0) => (lastOffset - firstOffsetVal + 1) @@ -157,7 +162,7 @@ class Log(@volatile var dir: File, @volatile var recoveryPoint: Long, scheduler: Scheduler, brokerTopicStats: BrokerTopicStats, - time: Time, + val time: Time, val maxProducerIdExpirationMs: Int, val producerIdExpirationCheckIntervalMs: Int, val topicPartition: TopicPartition, @@ -177,6 +182,10 @@ class Log(@volatile var dir: File, /* last time it was flushed */ private val lastFlushedTime = new AtomicLong(time.milliseconds) + def recordVersion: RecordVersion = config.messageFormatVersion.recordVersion + + def supportsLeaderEpoch = recordVersion.value >= RecordVersion.V2.value + def initFileSize: Int = { if (config.preallocate) config.segmentSize @@ -192,7 +201,15 @@ class Log(@volatile var dir: File, warn(s"${LogConfig.RetentionMsProp} for topic ${topicPartition.topic} is set to ${newConfig.retentionMs}. It is smaller than " + s"${LogConfig.MessageTimestampDifferenceMaxMsProp}'s value ${newConfig.messageTimestampDifferenceMaxMs}. " + s"This may result in frequent log rolling.") + val oldConfig = this.config this.config = newConfig + if (updatedKeys.contains(LogConfig.MessageFormatVersionProp)) { + val oldRecordVersion = oldConfig.messageFormatVersion.recordVersion + val newRecordVersion = newConfig.messageFormatVersion.recordVersion + if (newRecordVersion.precedes(oldRecordVersion)) + warn(s"Record format version has been downgraded from $oldRecordVersion to $newRecordVersion.") + _leaderEpochCache = initializeLeaderEpochCache() + } } private def checkIfMemoryMappedBufferClosed(): Unit = { @@ -225,7 +242,7 @@ class Log(@volatile var dir: File, /* the actual segments of the log */ private val segments: ConcurrentNavigableMap[java.lang.Long, LogSegment] = new ConcurrentSkipListMap[java.lang.Long, LogSegment] - @volatile private var _leaderEpochCache: LeaderEpochCache = initializeLeaderEpochCache() + @volatile private var _leaderEpochCache: LeaderEpochFileCache = initializeLeaderEpochCache() locally { val startMs = time.milliseconds @@ -235,13 +252,17 @@ class Log(@volatile var dir: File, /* Calculate the offset of the next message */ nextOffsetMetadata = new LogOffsetMetadata(nextOffset, activeSegment.baseOffset, activeSegment.size) - _leaderEpochCache.clearAndFlushLatest(nextOffsetMetadata.messageOffset) + _leaderEpochCache.truncateFromEnd(nextOffsetMetadata.messageOffset) logStartOffset = math.max(logStartOffset, segments.firstEntry.getValue.baseOffset) // The earliest leader epoch may not be flushed during a hard failure. Recover it here. - _leaderEpochCache.clearAndFlushEarliest(logStartOffset) + _leaderEpochCache.truncateFromStart(logStartOffset) + // Any segment loading or recovery code must not use producerStateManager, so that we can build the full state here + // from scratch. + if (!producerStateManager.isEmpty) + throw new IllegalStateException("Producer state must be empty during log initialization") loadProducerState(logEndOffset, reloadFromCleanShutdown = hasCleanShutdownFile) info(s"Completed load of log with ${segments.size} segments, log start offset $logStartOffset and " + @@ -288,49 +309,92 @@ class Log(@volatile var dir: File, def leaderEpochCache = _leaderEpochCache - private def initializeLeaderEpochCache(): LeaderEpochCache = { + private def initializeLeaderEpochCache(): LeaderEpochFileCache = { // create the log directory if it doesn't exist Files.createDirectories(dir.toPath) - new LeaderEpochFileCache(topicPartition, () => logEndOffsetMetadata, - new LeaderEpochCheckpointFile(LeaderEpochFile.newFile(dir), logDirFailureChannel)) + val checkpointFile = new LeaderEpochCheckpointFile(LeaderEpochFile.newFile(dir), logDirFailureChannel) + val cache = new LeaderEpochFileCache(topicPartition, logEndOffset _, checkpointFile) + + if (!supportsLeaderEpoch && cache.nonEmpty) { + warn(s"Clearing non-empty leader epoch cache due to incompatible message format $recordVersion") + cache.clearAndFlush() + } + cache } + /** + * Removes any temporary files found in log directory, and creates a list of all .swap files which could be swapped + * in place of existing segment(s). For log splitting, we know that any .swap file whose base offset is higher than + * the smallest offset .clean file could be part of an incomplete split operation. Such .swap files are also deleted + * by this method. + * @return Set of .swap files that are valid to be swapped in as segment files + */ private def removeTempFilesAndCollectSwapFiles(): Set[File] = { - def deleteIndicesIfExist(baseFile: File, swapFile: File, fileType: String): Unit = { - info(s"Found $fileType file ${swapFile.getAbsolutePath} from interrupted swap operation. Deleting index files (if they exist).") + def deleteIndicesIfExist(baseFile: File, suffix: String = ""): Unit = { + info(s"Deleting index files with suffix $suffix for baseFile $baseFile") val offset = offsetFromFile(baseFile) - Files.deleteIfExists(Log.offsetIndexFile(dir, offset).toPath) - Files.deleteIfExists(Log.timeIndexFile(dir, offset).toPath) - Files.deleteIfExists(Log.transactionIndexFile(dir, offset).toPath) + Files.deleteIfExists(Log.offsetIndexFile(dir, offset, suffix).toPath) + Files.deleteIfExists(Log.timeIndexFile(dir, offset, suffix).toPath) + Files.deleteIfExists(Log.transactionIndexFile(dir, offset, suffix).toPath) } var swapFiles = Set[File]() + var cleanFiles = Set[File]() + var minCleanedFileOffset = Long.MaxValue for (file <- dir.listFiles if file.isFile) { if (!file.canRead) throw new IOException(s"Could not read file $file") val filename = file.getName - if (filename.endsWith(DeletedFileSuffix) || filename.endsWith(CleanedFileSuffix)) { + if (filename.endsWith(DeletedFileSuffix)) { debug(s"Deleting stray temporary file ${file.getAbsolutePath}") Files.deleteIfExists(file.toPath) + } else if (filename.endsWith(CleanedFileSuffix)) { + minCleanedFileOffset = Math.min(offsetFromFileName(filename), minCleanedFileOffset) + cleanFiles += file } else if (filename.endsWith(SwapFileSuffix)) { // we crashed in the middle of a swap operation, to recover: // if a log, delete the index files, complete the swap operation later // if an index just delete the index files, they will be rebuilt val baseFile = new File(CoreUtils.replaceSuffix(file.getPath, SwapFileSuffix, "")) + info(s"Found file ${file.getAbsolutePath} from interrupted swap operation.") if (isIndexFile(baseFile)) { - deleteIndicesIfExist(baseFile, file, "index") + deleteIndicesIfExist(baseFile) } else if (isLogFile(baseFile)) { - deleteIndicesIfExist(baseFile, file, "log") + deleteIndicesIfExist(baseFile) swapFiles += file } } } - swapFiles + + // KAFKA-6264: Delete all .swap files whose base offset is greater than the minimum .cleaned segment offset. Such .swap + // files could be part of an incomplete split operation that could not complete. See Log#splitOverflowedSegment + // for more details about the split operation. + val (invalidSwapFiles, validSwapFiles) = swapFiles.partition(file => offsetFromFile(file) >= minCleanedFileOffset) + invalidSwapFiles.foreach { file => + debug(s"Deleting invalid swap file ${file.getAbsoluteFile} minCleanedFileOffset: $minCleanedFileOffset") + val baseFile = new File(CoreUtils.replaceSuffix(file.getPath, SwapFileSuffix, "")) + deleteIndicesIfExist(baseFile, SwapFileSuffix) + Files.deleteIfExists(file.toPath) + } + + // Now that we have deleted all .swap files that constitute an incomplete split operation, let's delete all .clean files + cleanFiles.foreach { file => + debug(s"Deleting stray .clean file ${file.getAbsolutePath}") + Files.deleteIfExists(file.toPath) + } + + validSwapFiles } - // This method does not need to convert IOException to KafkaStorageException because it is only called before all logs are loaded + /** + * This method does not need to convert IOException to KafkaStorageException because it is only called before all logs are loaded + * It is possible that we encounter a segment with index offset overflow in which case the LogSegmentOffsetOverflowException + * will be thrown. Note that any segments that were opened before we encountered the exception will remain open and the + * caller is responsible for closing them appropriately, if needed. + * @throws LogSegmentOffsetOverflowException if the log directory contains a segment with messages that overflow the index offset + */ private def loadSegmentFiles(): Unit = { // load segments in ascending order because transactional data from one segment may depend on the // segments that come before it @@ -369,30 +433,34 @@ class Log(@volatile var dir: File, } } - private def recoverSegment(segment: LogSegment, leaderEpochCache: Option[LeaderEpochCache] = None): Int = lock synchronized { - val stateManager = new ProducerStateManager(topicPartition, dir, maxProducerIdExpirationMs) - stateManager.truncateAndReload(logStartOffset, segment.baseOffset, time.milliseconds) - logSegments(stateManager.mapEndOffset, segment.baseOffset).foreach { segment => - val startOffset = math.max(segment.baseOffset, stateManager.mapEndOffset) - val fetchDataInfo = segment.read(startOffset, None, Int.MaxValue) - if (fetchDataInfo != null) - loadProducersFromLog(stateManager, fetchDataInfo.records) - } - stateManager.updateMapEndOffset(segment.baseOffset) - - // take a snapshot for the first recovered segment to avoid reloading all the segments if we shutdown before we - // checkpoint the recovery point - stateManager.takeSnapshot() - - val bytesTruncated = segment.recover(stateManager, leaderEpochCache) - + /** + * Recover the given segment. + * @param segment Segment to recover + * @param leaderEpochCache Optional cache for updating the leader epoch during recovery + * @return The number of bytes truncated from the segment + * @throws LogSegmentOffsetOverflowException if the segment contains messages that cause index offset overflow + */ + private def recoverSegment(segment: LogSegment, + leaderEpochCache: Option[LeaderEpochFileCache] = None): Int = lock synchronized { + val producerStateManager = new ProducerStateManager(topicPartition, dir, maxProducerIdExpirationMs) + rebuildProducerState(segment.baseOffset, reloadFromCleanShutdown = false, producerStateManager) + val bytesTruncated = segment.recover(producerStateManager, leaderEpochCache) // once we have recovered the segment's data, take a snapshot to ensure that we won't // need to reload the same segment again while recovering another segment. - stateManager.takeSnapshot() + producerStateManager.takeSnapshot() bytesTruncated } - // This method does not need to convert IOException to KafkaStorageException because it is only called before all logs are loaded + /** + * This method does not need to convert IOException to KafkaStorageException because it is only called before all logs + * are loaded. + * @throws LogSegmentOffsetOverflowException if the swap file contains messages that cause the log segment offset to + * overflow. Note that this is currently a fatal exception as we do not have + * a way to deal with it. The exception is propagated all the way up to + * KafkaServer#startup which will cause the broker to shut down if we are in + * this situation. This is expected to be an extremely rare scenario in practice, + * and manual intervention might be required to get out of it. + */ private def completeSwapOperations(swapFiles: Set[File]): Unit = { for (swapFile <- swapFiles) { val logFile = new File(CoreUtils.replaceSuffix(swapFile.getPath, SwapFileSuffix, "")) @@ -404,42 +472,71 @@ class Log(@volatile var dir: File, fileSuffix = SwapFileSuffix) info(s"Found log file ${swapFile.getPath} from interrupted swap operation, repairing.") recoverSegment(swapSegment) - val oldSegments = logSegments(swapSegment.baseOffset, swapSegment.readNextOffset) - replaceSegments(swapSegment, oldSegments.toSeq, isRecoveredSwapFile = true) + + // We create swap files for two cases: + // (1) Log cleaning where multiple segments are merged into one, and + // (2) Log splitting where one segment is split into multiple. + // + // Both of these mean that the resultant swap segments be composed of the original set, i.e. the swap segment + // must fall within the range of existing segment(s). If we cannot find such a segment, it means the deletion + // of that segment was successful. In such an event, we should simply rename the .swap to .log without having to + // do a replace with an existing segment. + val oldSegments = logSegments(swapSegment.baseOffset, swapSegment.readNextOffset).filter { segment => + segment.readNextOffset > swapSegment.baseOffset + } + replaceSegments(Seq(swapSegment), oldSegments.toSeq, isRecoveredSwapFile = true) } } - // Load the log segments from the log files on disk and return the next offset - // This method does not need to convert IOException to KafkaStorageException because it is only called before all logs are loaded + /** + * Load the log segments from the log files on disk and return the next offset. + * This method does not need to convert IOException to KafkaStorageException because it is only called before all logs + * are loaded. + * @throws LogSegmentOffsetOverflowException if we encounter a .swap file with messages that overflow index offset; or when + * we find an unexpected number of .log files with overflow + */ private def loadSegments(): Long = { // first do a pass through the files in the log directory and remove any temporary files // and find any interrupted swap operations val swapFiles = removeTempFilesAndCollectSwapFiles() - // now do a second pass and load all the log and index files - loadSegmentFiles() + // Now do a second pass and load all the log and index files. + // We might encounter legacy log segments with offset overflow (KAFKA-6264). We need to split such segments. When + // this happens, restart loading segment files from scratch. + retryOnOffsetOverflow { + // In case we encounter a segment with offset overflow, the retry logic will split it after which we need to retry + // loading of segments. In that case, we also need to close all segments that could have been left open in previous + // call to loadSegmentFiles(). + logSegments.foreach(_.close()) + segments.clear() + loadSegmentFiles() + } // 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. completeSwapOperations(swapFiles) - if (logSegments.isEmpty) { - // no existing segments, create a new mutable segment beginning at offset 0 - addSegment(LogSegment.open(dir = dir, - baseOffset = 0, - config, - time = time, - fileAlreadyExists = false, - initFileSize = this.initFileSize, - preallocate = config.preallocate)) - 0 - } else if (!dir.getAbsolutePath.endsWith(Log.DeleteDirSuffix)) { - val nextOffset = recoverLog() + if (!dir.getAbsolutePath.endsWith(Log.DeleteDirSuffix)) { + val nextOffset = retryOnOffsetOverflow { + recoverLog() + } + // reset the index size of the currently active log segment to allow more entries activeSegment.resizeIndexes(config.maxIndexSize) nextOffset - } else 0 + } else { + if (logSegments.isEmpty) { + addSegment(LogSegment.open(dir = dir, + baseOffset = 0, + config, + time = time, + fileAlreadyExists = false, + initFileSize = this.initFileSize, + preallocate = false)) + } + 0 + } } private def updateLogEndOffset(messageOffset: Long) { @@ -448,9 +545,9 @@ class Log(@volatile var dir: File, /** * Recover the log segments and return the next offset after recovery. - * * This method does not need to convert IOException to KafkaStorageException because it is only called before all * logs are loaded. + * @throws LogSegmentOffsetOverflowException if we encountered a legacy segment with offset overflow */ private def recoverLog(): Long = { // if we have the clean shutdown marker, skip recovery @@ -462,7 +559,7 @@ class Log(@volatile var dir: File, info(s"Recovering unflushed segment ${segment.baseOffset}") val truncatedBytes = try { - recoverSegment(segment, Some(_leaderEpochCache)) + recoverSegment(segment, if (supportsLeaderEpoch) Some(_leaderEpochCache) else None) } catch { case _: InvalidOffsetException => val startOffset = segment.baseOffset @@ -477,14 +574,47 @@ class Log(@volatile var dir: File, } } } + + if (logSegments.nonEmpty) { + val logEndOffset = activeSegment.readNextOffset + if (logEndOffset < logStartOffset) { + warn(s"Deleting all segments because logEndOffset ($logEndOffset) is smaller than logStartOffset ($logStartOffset). " + + "This could happen if segment files were deleted from the file system.") + logSegments.foreach(deleteSegment) + } + } + + if (logSegments.isEmpty) { + // no existing segments, create a new mutable segment beginning at logStartOffset + addSegment(LogSegment.open(dir = dir, + baseOffset = logStartOffset, + config, + time = time, + fileAlreadyExists = false, + initFileSize = this.initFileSize, + preallocate = config.preallocate)) + } + recoveryPoint = activeSegment.readNextOffset recoveryPoint } - private def loadProducerState(lastOffset: Long, reloadFromCleanShutdown: Boolean): Unit = lock synchronized { + // Rebuild producer state until lastOffset. This method may be called from the recovery code path, and thus must be + // free of all side-effects, i.e. it must not update any log-specific state. + private def rebuildProducerState(lastOffset: Long, + reloadFromCleanShutdown: Boolean, + producerStateManager: ProducerStateManager): Unit = lock synchronized { checkIfMemoryMappedBufferClosed() val messageFormatVersion = config.messageFormatVersion.recordVersion.value - info(s"Loading producer state from offset $lastOffset with message format version $messageFormatVersion") + val segments = logSegments + val offsetsToSnapshot = + if (segments.nonEmpty) { + val nextLatestSegmentBaseOffset = lowerSegment(segments.last.baseOffset).map(_.baseOffset) + Seq(nextLatestSegmentBaseOffset, Some(segments.last.baseOffset), Some(lastOffset)) + } else { + Seq(Some(lastOffset)) + } + info(s"Loading producer state till offset $lastOffset with message format version $messageFormatVersion") // We want to avoid unnecessary scanning of the log to build the producer state when the broker is being // upgraded. The basic idea is to use the absence of producer snapshot files to detect the upgrade case, @@ -498,13 +628,11 @@ class Log(@volatile var dir: File, // offset (see below). The next time the log is reloaded, we will load producer state using this snapshot // (or later snapshots). Otherwise, if there is no snapshot file, then we have to rebuild producer state // from the first segment. - - if (producerStateManager.latestSnapshotOffset.isEmpty && (messageFormatVersion < RecordBatch.MAGIC_VALUE_V2 || reloadFromCleanShutdown)) { + if (messageFormatVersion < RecordBatch.MAGIC_VALUE_V2 || + (producerStateManager.latestSnapshotOffset.isEmpty && reloadFromCleanShutdown)) { // To avoid an expensive scan through all of the segments, we take empty snapshots from the start of the // last two segments and the last offset. This should avoid the full scan in the case that the log needs // truncation. - val nextLatestSegmentBaseOffset = lowerSegment(activeSegment.baseOffset).map(_.baseOffset) - val offsetsToSnapshot = Seq(nextLatestSegmentBaseOffset, Some(activeSegment.baseOffset), Some(lastOffset)) offsetsToSnapshot.flatten.foreach { offset => producerStateManager.updateMapEndOffset(offset) producerStateManager.takeSnapshot() @@ -523,19 +651,25 @@ class Log(@volatile var dir: File, logSegments(producerStateManager.mapEndOffset, lastOffset).foreach { segment => val startOffset = Utils.max(segment.baseOffset, producerStateManager.mapEndOffset, logStartOffset) producerStateManager.updateMapEndOffset(startOffset) - producerStateManager.takeSnapshot() + + if (offsetsToSnapshot.contains(Some(segment.baseOffset))) + producerStateManager.takeSnapshot() val fetchDataInfo = segment.read(startOffset, Some(lastOffset), Int.MaxValue) if (fetchDataInfo != null) loadProducersFromLog(producerStateManager, fetchDataInfo.records) } } - producerStateManager.updateMapEndOffset(lastOffset) - updateFirstUnstableOffset() + producerStateManager.takeSnapshot() } } + private def loadProducerState(lastOffset: Long, reloadFromCleanShutdown: Boolean): Unit = lock synchronized { + rebuildProducerState(lastOffset, reloadFromCleanShutdown, producerStateManager) + updateFirstUnstableOffset() + } + private def loadProducersFromLog(producerStateManager: ProducerStateManager, records: Records): Unit = { val loadedProducers = mutable.Map.empty[Long, ProducerAppendInfo] val completedTxns = ListBuffer.empty[CompletedTxn] @@ -555,6 +689,14 @@ class Log(@volatile var dir: File, } } + private[log] def lastRecordsOfActiveProducers: Map[Long, LastRecord] = lock synchronized { + producerStateManager.activeProducers.map { case (producerId, producerIdEntry) => + val lastDataOffset = if (producerIdEntry.lastDataOffset >= 0 ) Some(producerIdEntry.lastDataOffset) else None + val lastRecord = LastRecord(lastDataOffset, producerIdEntry.producerEpoch) + producerId -> lastRecord + } + } + /** * Check if we have the "clean shutdown" file */ @@ -585,10 +727,10 @@ class Log(@volatile var dir: File, } /** - * Rename the directory of the log - * - * @throws KafkaStorageException if rename fails - */ + * Rename the directory of the log + * + * @throws KafkaStorageException if rename fails + */ def renameDir(name: String) { lock synchronized { maybeHandleIOException(s"Error while renaming dir for $topicPartition in log dir ${dir.getParent}") { @@ -651,6 +793,8 @@ class Log(@volatile var dir: File, * @param assignOffsets Should the log assign offsets to this message set or blindly apply what it is given * @param leaderEpoch The partition's leader epoch which will be applied to messages when offsets are assigned on the leader * @throws KafkaStorageException If the append fails due to an I/O error. + * @throws OffsetsOutOfOrderException If out of order offsets found in 'records' + * @throws UnexpectedAppendOffsetException If the first or last offset in append is less than next offset * @return Information about the appended messages including the first and last offset. */ private def append(records: MemoryRecords, isFromClient: Boolean, assignOffsets: Boolean, leaderEpoch: Int): LogAppendInfo = { @@ -713,9 +857,27 @@ class Log(@volatile var dir: File, } } else { // we are taking the offsets we are given - if (!appendInfo.offsetsMonotonic || appendInfo.firstOrLastOffset < nextOffsetMetadata.messageOffset) - throw new IllegalArgumentException(s"Out of order offsets found in append to $topicPartition: " + - records.records.asScala.map(_.offset)) + if (!appendInfo.offsetsMonotonic) + throw new OffsetsOutOfOrderException(s"Out of order offsets found in append to $topicPartition: " + + records.records.asScala.map(_.offset)) + + if (appendInfo.firstOrLastOffsetOfFirstBatch < nextOffsetMetadata.messageOffset) { + // we may still be able to recover if the log is empty + // one example: fetching from log start offset on the leader which is not batch aligned, + // which may happen as a result of AdminClient#deleteRecords() + val firstOffset = appendInfo.firstOffset match { + case Some(offset) => offset + case None => records.batches.asScala.head.baseOffset() + } + + val firstOrLast = if (appendInfo.firstOffset.isDefined) "First offset" else "Last offset of the first batch" + throw new UnexpectedAppendOffsetException( + s"Unexpected offset in append to $topicPartition. $firstOrLast " + + s"${appendInfo.firstOrLastOffsetOfFirstBatch} is less than the next offset ${nextOffsetMetadata.messageOffset}. " + + s"First 10 offsets in append: ${records.records.asScala.take(10).map(_.offset)}, last offset in" + + s" append: ${appendInfo.lastOffset}. Log start offset = $logStartOffset", + firstOffset, appendInfo.lastOffset) + } } // update the epoch cache with the epoch stamped onto the message by the leader @@ -745,7 +907,7 @@ class Log(@volatile var dir: File, val segment = maybeRoll(validRecords.sizeInBytes, appendInfo) val logOffsetMetadata = LogOffsetMetadata( - messageOffset = appendInfo.firstOrLastOffset, + messageOffset = appendInfo.firstOrLastOffsetOfFirstBatch, segmentBaseOffset = segment.baseOffset, relativePositionInSegment = segment.size) @@ -754,6 +916,14 @@ class Log(@volatile var dir: File, shallowOffsetOfMaxTimestamp = appendInfo.offsetOfMaxTimestamp, records = validRecords) + // Increment the log end offset. We do this immediately after the append because a + // write to the transaction index below may fail and we want to ensure that the offsets + // of future appends still grow monotonically. The resulting transaction index inconsistency + // will be cleaned up after the log directory is recovered. Note that the end offset of the + // ProducerStateManager will not be updated and the last stable offset will not advance + // if the append to the transaction index fails. + updateLogEndOffset(appendInfo.lastOffset + 1) + // update the producer state for ((_, producerAppendInfo) <- updatedProducers) { producerAppendInfo.maybeCacheTxnFirstOffsetMetadata(logOffsetMetadata) @@ -763,17 +933,15 @@ class Log(@volatile var dir: File, // update the transaction index with the true last stable offset. The last offset visible // to consumers using READ_COMMITTED will be limited by this value and the high watermark. for (completedTxn <- completedTxns) { - val lastStableOffset = producerStateManager.completeTxn(completedTxn) + val lastStableOffset = producerStateManager.lastStableOffset(completedTxn) segment.updateTxnIndex(completedTxn, lastStableOffset) + producerStateManager.completeTxn(completedTxn) } // always update the last producer id map offset so that the snapshot reflects the current offset // even if there isn't any idempotent data being written producerStateManager.updateMapEndOffset(appendInfo.lastOffset + 1) - // increment the log end offset - updateLogEndOffset(appendInfo.lastOffset + 1) - // update the first unstable offset (which is used to compute LSO) updateFirstUnstableOffset() @@ -828,7 +996,7 @@ class Log(@volatile var dir: File, if (newLogStartOffset > logStartOffset) { info(s"Incrementing log start offset to $newLogStartOffset") logStartOffset = newLogStartOffset - _leaderEpochCache.clearAndFlushEarliest(logStartOffset) + _leaderEpochCache.truncateFromStart(logStartOffset) producerStateManager.truncateHead(logStartOffset) updateFirstUnstableOffset() } @@ -885,6 +1053,7 @@ class Log(@volatile var dir: File, var maxTimestamp = RecordBatch.NO_TIMESTAMP var offsetOfMaxTimestamp = -1L var readFirstMessage = false + var lastOffsetOfFirstBatch = -1L for (batch <- records.batches.asScala) { // we only validate V2 and higher to avoid potential compatibility issues with older clients @@ -901,6 +1070,7 @@ class Log(@volatile var dir: File, if (!readFirstMessage) { if (batch.magic >= RecordBatch.MAGIC_VALUE_V2) firstOffset = Some(batch.baseOffset) + lastOffsetOfFirstBatch = batch.lastOffset readFirstMessage = true } @@ -939,7 +1109,7 @@ class Log(@volatile var dir: File, // Apply broker-side compression if any val targetCodec = BrokerCompressionCodec.getTargetCompressionCodec(config.compressionType, sourceCodec) LogAppendInfo(firstOffset, lastOffset, maxTimestamp, offsetOfMaxTimestamp, RecordBatch.NO_TIMESTAMP, logStartOffset, - RecordConversionStats.EMPTY, sourceCodec, targetCodec, shallowMessageCount, validBytesCount, monotonic) + RecordConversionStats.EMPTY, sourceCodec, targetCodec, shallowMessageCount, validBytesCount, monotonic, lastOffsetOfFirstBatch) } private def updateProducers(batch: RecordBatch, @@ -1315,9 +1485,9 @@ class Log(@volatile var dir: File, if (segment.shouldRoll(messagesSize, maxTimestampInMessages, maxOffsetInMessages, now)) { debug(s"Rolling new log segment (log_size = ${segment.size}/${config.segmentSize}}, " + - s"offset_index_size = ${segment.offsetIndex.entries}/${segment.offsetIndex.maxEntries}, " + - s"time_index_size = ${segment.timeIndex.entries}/${segment.timeIndex.maxEntries}, " + - s"inactive_time_ms = ${segment.timeWaitedForRoll(now, maxTimestampInMessages)}/${config.segmentMs - segment.rollJitterMs}).") + s"offset_index_size = ${segment.offsetIndex.entries}/${segment.offsetIndex.maxEntries}, " + + s"time_index_size = ${segment.timeIndex.entries}/${segment.timeIndex.maxEntries}, " + + s"inactive_time_ms = ${segment.timeWaitedForRoll(now, maxTimestampInMessages)}/${config.segmentMs - segment.rollJitterMs}).") /* maxOffsetInMessages - Integer.MAX_VALUE is a heuristic value for the first offset in the set of messages. @@ -1332,8 +1502,8 @@ class Log(@volatile var dir: File, in the header. */ appendInfo.firstOffset match { - case Some(firstOffset) => roll(firstOffset) - case None => roll(maxOffsetInMessages - Integer.MAX_VALUE) + case Some(firstOffset) => roll(Some(firstOffset)) + case None => roll(Some(maxOffsetInMessages - Integer.MAX_VALUE)) } } else { segment @@ -1346,22 +1516,45 @@ class Log(@volatile var dir: File, * * @return The newly rolled segment */ - def roll(expectedNextOffset: Long = 0): LogSegment = { + def roll(expectedNextOffset: Option[Long] = None): LogSegment = { maybeHandleIOException(s"Error while rolling log segment for $topicPartition in dir ${dir.getParent}") { val start = time.hiResClockMs() lock synchronized { checkIfMemoryMappedBufferClosed() - val newOffset = math.max(expectedNextOffset, logEndOffset) + val newOffset = math.max(expectedNextOffset.getOrElse(0L), logEndOffset) val logFile = Log.logFile(dir, newOffset) - val offsetIdxFile = offsetIndexFile(dir, newOffset) - val timeIdxFile = timeIndexFile(dir, newOffset) - val txnIdxFile = transactionIndexFile(dir, newOffset) - for (file <- List(logFile, offsetIdxFile, timeIdxFile, txnIdxFile) if file.exists) { - warn(s"Newly rolled segment file ${file.getAbsolutePath} already exists; deleting it first") - Files.delete(file.toPath) - } - Option(segments.lastEntry).foreach(_.getValue.onBecomeInactiveSegment()) + if (segments.containsKey(newOffset)) { + // segment with the same base offset already exists and loaded + if (activeSegment.baseOffset == newOffset && activeSegment.size == 0) { + // We have seen this happen (see KAFKA-6388) after shouldRoll() returns true for an + // active segment of size zero because of one of the indexes is "full" (due to _maxEntries == 0). + warn(s"Trying to roll a new log segment with start offset $newOffset " + + s"=max(provided offset = $expectedNextOffset, LEO = $logEndOffset) while it already " + + s"exists and is active with size 0. Size of time index: ${activeSegment.timeIndex.entries}," + + s" size of offset index: ${activeSegment.offsetIndex.entries}.") + deleteSegment(activeSegment) + } else { + throw new KafkaException(s"Trying to roll a new log segment for topic partition $topicPartition with start offset $newOffset" + + s" =max(provided offset = $expectedNextOffset, LEO = $logEndOffset) while it already exists. Existing " + + s"segment is ${segments.get(newOffset)}.") + } + } else if (!segments.isEmpty && newOffset < activeSegment.baseOffset) { + throw new KafkaException( + s"Trying to roll a new log segment for topic partition $topicPartition with " + + s"start offset $newOffset =max(provided offset = $expectedNextOffset, LEO = $logEndOffset) lower than start offset of the active segment $activeSegment") + } else { + val offsetIdxFile = offsetIndexFile(dir, newOffset) + val timeIdxFile = timeIndexFile(dir, newOffset) + val txnIdxFile = transactionIndexFile(dir, newOffset) + + for (file <- List(logFile, offsetIdxFile, timeIdxFile, txnIdxFile) if file.exists) { + warn(s"Newly rolled segment file ${file.getAbsolutePath} already exists; deleting it first") + Files.delete(file.toPath) + } + + Option(segments.lastEntry).foreach(_.getValue.onBecomeInactiveSegment()) + } // take a snapshot of the producer state to facilitate recovery. It is useful to have the snapshot // offset align with the new segment offset since this ensures we can recover the segment by beginning @@ -1378,10 +1571,7 @@ class Log(@volatile var dir: File, fileAlreadyExists = false, initFileSize = initFileSize, preallocate = config.preallocate) - val prev = addSegment(segment) - if (prev != null) - throw new KafkaException(s"Trying to roll a new log segment for topic partition $topicPartition with " + - s"start offset $newOffset while it already exists.") + addSegment(segment) // 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) @@ -1530,7 +1720,7 @@ class Log(@volatile var dir: File, updateLogEndOffset(targetOffset) this.recoveryPoint = math.min(targetOffset, this.recoveryPoint) this.logStartOffset = math.min(targetOffset, this.logStartOffset) - _leaderEpochCache.clearAndFlushLatest(targetOffset) + _leaderEpochCache.truncateFromEnd(targetOffset) loadProducerState(targetOffset, reloadFromCleanShutdown = false) } true @@ -1625,7 +1815,9 @@ class Log(@volatile var dir: File, } /** - * Perform an asynchronous delete on the given file if it exists (otherwise do nothing) + * Perform an asynchronous delete on the given file. + * + * This method assumes that the file exists and the method is not thread-safe. * * This method does not need to convert IOException (thrown from changeFileSuffixes) to KafkaStorageException because * it is either called before all logs are loaded or the caller will catch and handle IOException @@ -1644,51 +1836,62 @@ class Log(@volatile var 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. + * Swap one or more new segment in place and delete one or more existing segments in a crash-safe manner. The old + * segments will be asynchronously deleted. * * This method does not need to convert IOException to KafkaStorageException because it is either called before all logs are loaded * or the caller will catch and handle IOException * * The sequence of operations is: *
    - *
  1. Cleaner creates new segment with suffix .cleaned and invokes replaceSegments(). + *
  2. Cleaner creates one or more new segments 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(). - *
  3. 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. + * the .cleaned files are deleted on recovery in loadSegments(). + *
  4. New segments are renamed .swap. If the broker crashes before all segments were renamed to .swap, the + * clean-and-swap operation is aborted - .cleaned as well as .swap files are deleted on recovery in + * loadSegments(). We detect this situation by maintaining a specific order in which files are renamed from + * .cleaned to .swap. Basically, files are renamed in descending order of offsets. On recovery, all .swap files + * whose offset is greater than the minimum-offset .clean file are deleted. + *
  5. If the broker crashes after all new segments were renamed to .swap, the operation is completed, the swap + * operation is resumed on recovery as described in the next step. *
  6. 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. - *
  7. Swap segment is renamed to replace the existing segment, completing this operation. + *
  8. Swap segment(s) are renamed to replace the existing segments, 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 newSegments 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], isRecoveredSwapFile: Boolean = false) { + private[log] def replaceSegments(newSegments: Seq[LogSegment], oldSegments: Seq[LogSegment], isRecoveredSwapFile: Boolean = false) { lock synchronized { + val sortedNewSegments = newSegments.sortBy(_.baseOffset) + // Some old segments may have been removed from index and scheduled for async deletion after the caller reads segments + // but before this method is executed. We want to filter out those segments to avoid calling asyncDeleteSegment() + // multiple times for the same segment. + val sortedOldSegments = oldSegments.filter(seg => segments.containsKey(seg.baseOffset)).sortBy(_.baseOffset) + checkIfMemoryMappedBufferClosed() // 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() if (!isRecoveredSwapFile) - newSegment.changeFileSuffixes(Log.CleanedFileSuffix, Log.SwapFileSuffix) - addSegment(newSegment) + sortedNewSegments.reverse.foreach(_.changeFileSuffixes(Log.CleanedFileSuffix, Log.SwapFileSuffix)) + sortedNewSegments.reverse.foreach(addSegment(_)) // delete the old files - for (seg <- oldSegments) { + for (seg <- sortedOldSegments) { // remove the index entry - if (seg.baseOffset != newSegment.baseOffset) + if (seg.baseOffset != sortedNewSegments.head.baseOffset) segments.remove(seg.baseOffset) // delete segment asyncDeleteSegment(seg) } // okay we are safe now, remove the swap suffix - newSegment.changeFileSuffixes(Log.SwapFileSuffix, "") + sortedNewSegments.foreach(_.changeFileSuffixes(Log.SwapFileSuffix, "")) } } @@ -1701,12 +1904,13 @@ class Log(@volatile var dir: File, 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 */ - def addSegment(segment: LogSegment) = this.segments.put(segment.baseOffset, segment) + @threadsafe + def addSegment(segment: LogSegment): LogSegment = this.segments.put(segment.baseOffset, segment) private def maybeHandleIOException[T](msg: => String)(fun: => T): T = { try { @@ -1718,6 +1922,81 @@ class Log(@volatile var dir: File, } } + private[log] def retryOnOffsetOverflow[T](fn: => T): T = { + while (true) { + try { + return fn + } catch { + case e: LogSegmentOffsetOverflowException => + info(s"Caught segment overflow error: ${e.getMessage}. Split segment and retry.") + splitOverflowedSegment(e.segment) + } + } + throw new IllegalStateException() + } + + /** + * Split a segment into one or more segments such that there is no offset overflow in any of them. The + * resulting segments will contain the exact same messages that are present in the input segment. On successful + * completion of this method, the input segment will be deleted and will be replaced by the resulting new segments. + * See replaceSegments for recovery logic, in case the broker dies in the middle of this operation. + *

Note that this method assumes we have already determined that the segment passed in contains records that cause + * offset overflow.

+ *

The split logic overloads the use of .clean files that LogCleaner typically uses to make the process of replacing + * the input segment with multiple new segments atomic and recoverable in the event of a crash. See replaceSegments + * and completeSwapOperations for the implementation to make this operation recoverable on crashes.

+ * @param segment Segment to split + * @return List of new segments that replace the input segment + */ + private[log] def splitOverflowedSegment(segment: LogSegment): List[LogSegment] = { + require(isLogFile(segment.log.file), s"Cannot split file ${segment.log.file.getAbsoluteFile}") + require(segment.hasOverflow, "Split operation is only permitted for segments with overflow") + + info(s"Splitting overflowed segment $segment") + + val newSegments = ListBuffer[LogSegment]() + try { + var position = 0 + val sourceRecords = segment.log + + while (position < sourceRecords.sizeInBytes) { + val firstBatch = sourceRecords.batchesFrom(position).asScala.head + val newSegment = LogCleaner.createNewCleanedSegment(this, firstBatch.baseOffset) + newSegments += newSegment + + val bytesAppended = newSegment.appendFromFile(sourceRecords, position) + if (bytesAppended == 0) + throw new IllegalStateException(s"Failed to append records from position $position in $segment") + + position += bytesAppended + } + + // prepare new segments + var totalSizeOfNewSegments = 0 + newSegments.foreach { splitSegment => + splitSegment.onBecomeInactiveSegment() + splitSegment.flush() + splitSegment.lastModified = segment.lastModified + totalSizeOfNewSegments += splitSegment.log.sizeInBytes + } + // size of all the new segments combined must equal size of the original segment + if (totalSizeOfNewSegments != segment.log.sizeInBytes) + throw new IllegalStateException("Inconsistent segment sizes after split" + + s" before: ${segment.log.sizeInBytes} after: $totalSizeOfNewSegments") + + // replace old segment with new ones + info(s"Replacing overflowed segment $segment with split segments $newSegments") + replaceSegments(newSegments.toList, List(segment), isRecoveredSwapFile = false) + newSegments.toList + } catch { + case e: Exception => + newSegments.foreach { splitSegment => + splitSegment.close() + splitSegment.deleteIfExists() + } + throw e + } + } } /** @@ -1809,17 +2088,17 @@ object Log { new File(dir, filenamePrefixFromOffset(offset) + LogFileSuffix + suffix) /** - * Return a directory name to rename the log directory to for async deletion. The name will be in the following - * format: topic-partition.uniqueId-delete where topic, partition and uniqueId are variables. - */ + * Return a directory name to rename the log directory to for async deletion. The name will be in the following + * format: topic-partition.uniqueId-delete where topic, partition and uniqueId are variables. + */ def logDeleteDirName(topicPartition: TopicPartition): String = { logDirNameWithSuffix(topicPartition, DeleteDirSuffix) } /** - * Return a future directory name for the given topic partition. The name will be in the following - * format: topic-partition.uniqueId-future where topic, partition and uniqueId are variables. - */ + * Return a future directory name for the given topic partition. The name will be in the following + * format: topic-partition.uniqueId-future where topic, partition and uniqueId are variables. + */ def logFutureDirName(topicPartition: TopicPartition): String = { logDirNameWithSuffix(topicPartition, FutureDirSuffix) } @@ -1830,9 +2109,9 @@ object Log { } /** - * Return a directory name for the given topic partition. The name will be in the following - * format: topic-partition where topic, partition are variables. - */ + * Return a directory name for the given topic partition. The name will be in the following + * format: topic-partition where topic, partition are variables. + */ def logDirName(topicPartition: TopicPartition): String = { s"${topicPartition.topic}-${topicPartition.partition}" } @@ -1857,6 +2136,9 @@ object Log { def timeIndexFile(dir: File, offset: Long, suffix: String = ""): File = new File(dir, filenamePrefixFromOffset(offset) + TimeIndexFileSuffix + suffix) + def deleteFileIfExists(file: File, suffix: String = ""): Unit = + Files.deleteIfExists(new File(file.getPath + suffix).toPath) + /** * Construct a producer id snapshot file using the given offset. * @@ -1876,17 +2158,20 @@ object Log { def transactionIndexFile(dir: File, offset: Long, suffix: String = ""): File = new File(dir, filenamePrefixFromOffset(offset) + TxnIndexFileSuffix + suffix) - def offsetFromFile(file: File): Long = { - val filename = file.getName + def offsetFromFileName(filename: String): Long = { filename.substring(0, filename.indexOf('.')).toLong } + def offsetFromFile(file: File): Long = { + offsetFromFileName(file.getName) + } + /** - * Calculate a log's size (in bytes) based on its log segments - * - * @param segments The log segments to calculate the size of - * @return Sum of the log segments' sizes (in bytes) - */ + * Calculate a log's size (in bytes) based on its log segments + * + * @param segments The log segments to calculate the size of + * @return Sum of the log segments' sizes (in bytes) + */ def sizeInBytes(segments: Iterable[LogSegment]): Long = segments.map(_.size.toLong).sum diff --git a/core/src/main/scala/kafka/log/LogCleaner.scala b/core/src/main/scala/kafka/log/LogCleaner.scala index aa7cfe276c407..36d64f4d6cdd1 100644 --- a/core/src/main/scala/kafka/log/LogCleaner.scala +++ b/core/src/main/scala/kafka/log/LogCleaner.scala @@ -19,7 +19,6 @@ package kafka.log import java.io.{File, IOException} import java.nio._ -import java.nio.file.Files import java.util.Date import java.util.concurrent.TimeUnit @@ -28,16 +27,16 @@ import kafka.common._ import kafka.metrics.KafkaMetricsGroup import kafka.server.{BrokerReconfigurable, KafkaConfig, LogDirFailureChannel} import kafka.utils._ -import org.apache.kafka.common.record._ -import org.apache.kafka.common.utils.Time -import org.apache.kafka.common.TopicPartition +import org.apache.kafka.common.{KafkaException, TopicPartition} import org.apache.kafka.common.config.ConfigException import org.apache.kafka.common.errors.{CorruptRecordException, KafkaStorageException} import org.apache.kafka.common.record.MemoryRecords.RecordFilter import org.apache.kafka.common.record.MemoryRecords.RecordFilter.BatchRetention +import org.apache.kafka.common.record._ +import org.apache.kafka.common.utils.Time -import scala.collection.{Set, mutable} import scala.collection.JavaConverters._ +import scala.collection.{Set, mutable} /** * The cleaner is responsible for removing obsolete records from logs which have the "compact" retention strategy. @@ -382,6 +381,12 @@ object LogCleaner { enableCleaner = config.logCleanerEnable) } + + def createNewCleanedSegment(log: Log, baseOffset: Long): LogSegment = { + LogSegment.deleteIfExists(log.dir, baseOffset, fileSuffix = Log.CleanedFileSuffix) + LogSegment.open(log.dir, baseOffset, log.config, Time.SYSTEM, fileAlreadyExists = false, + fileSuffix = Log.CleanedFileSuffix, initFileSize = log.initFileSize, preallocate = log.config.preallocate) + } } /** @@ -454,7 +459,6 @@ private[log] class Cleaner(val id: Int, // this is the lower of the last active segment and the compaction lag val cleanableHorizonMs = log.logSegments(0, cleanable.firstUncleanableOffset).lastOption.map(_.lastModified).getOrElse(0L) - // group the segments and clean the groups info("Cleaning log %s (cleaning prior to %s, discarding tombstones prior to %s)...".format(log.name, new Date(cleanableHorizonMs), new Date(deleteHorizonMs))) for (group <- groupSegmentsBySize(log.logSegments(0, endOffset), log.config.segmentSize, log.config.maxIndexSize, cleanable.firstUncleanableOffset)) @@ -482,26 +486,15 @@ private[log] class Cleaner(val id: Int, map: OffsetMap, deleteHorizonMs: Long, stats: CleanerStats) { - - def deleteCleanedFileIfExists(file: File): Unit = { - Files.deleteIfExists(new File(file.getPath + Log.CleanedFileSuffix).toPath) - } - // create a new segment with a suffix appended to the name of the log and indexes - val firstSegment = segments.head - deleteCleanedFileIfExists(firstSegment.log.file) - deleteCleanedFileIfExists(firstSegment.offsetIndex.file) - deleteCleanedFileIfExists(firstSegment.timeIndex.file) - deleteCleanedFileIfExists(firstSegment.txnIndex.file) - - val baseOffset = firstSegment.baseOffset - val cleaned = LogSegment.open(log.dir, baseOffset, log.config, time, fileSuffix = Log.CleanedFileSuffix, - initFileSize = log.initFileSize, preallocate = log.config.preallocate) + val cleaned = LogCleaner.createNewCleanedSegment(log, segments.head.baseOffset) try { // clean segments into the new destination segment val iter = segments.iterator var currentSegmentOpt: Option[LogSegment] = Some(iter.next()) + val lastOffsetOfActiveProducers = log.lastRecordsOfActiveProducers + while (currentSegmentOpt.isDefined) { val currentSegment = currentSegmentOpt.get val nextSegmentOpt = if (iter.hasNext) Some(iter.next()) else None @@ -514,9 +507,18 @@ private[log] class Cleaner(val id: Int, val retainDeletes = currentSegment.lastModified > deleteHorizonMs info(s"Cleaning segment $startOffset in log ${log.name} (largest timestamp ${new Date(currentSegment.largestTimestamp)}) " + s"into ${cleaned.baseOffset}, ${if(retainDeletes) "retaining" else "discarding"} deletes.") - cleanInto(log.topicPartition, currentSegment.log, cleaned, map, retainDeletes, log.config.maxMessageSize, - transactionMetadata, log.activeProducersWithLastSequence, stats) + try { + cleanInto(log.topicPartition, currentSegment.log, cleaned, map, retainDeletes, log.config.maxMessageSize, + transactionMetadata, lastOffsetOfActiveProducers, stats) + } catch { + case e: LogSegmentOffsetOverflowException => + // Split the current segment. It's also safest to abort the current cleaning process, so that we retry from + // scratch once the split is complete. + info(s"Caught segment overflow error during cleaning: ${e.getMessage}") + log.splitOverflowedSegment(currentSegment) + throw new LogCleaningAbortedException() + } currentSegmentOpt = nextSegmentOpt } @@ -529,9 +531,8 @@ private[log] class Cleaner(val id: Int, cleaned.lastModified = modified // swap in new segment - info(s"Swapping in cleaned segment ${cleaned.baseOffset} for segment(s) ${segments.map(_.baseOffset).mkString(",")} " + - s"in log ${log.name}") - log.replaceSegments(cleaned, segments) + info(s"Swapping in cleaned segment $cleaned for segment(s) $segments in log $log") + log.replaceSegments(List(cleaned), segments) } catch { case e: LogCleaningAbortedException => try cleaned.deleteIfExists() @@ -561,9 +562,9 @@ private[log] class Cleaner(val id: Int, retainDeletes: Boolean, maxLogMessageSize: Int, transactionMetadata: CleanedTransactionMetadata, - activeProducers: Map[Long, Int], + lastRecordsOfActiveProducers: Map[Long, LastRecord], stats: CleanerStats) { - val logCleanerFilter = new RecordFilter { + val logCleanerFilter: RecordFilter = new RecordFilter { var discardBatchRecords: Boolean = _ override def checkBatchRetention(batch: RecordBatch): BatchRetention = { @@ -571,9 +572,22 @@ private[log] class Cleaner(val id: Int, // note that we will never delete a marker until all the records from that transaction are removed. discardBatchRecords = shouldDiscardBatch(batch, transactionMetadata, retainTxnMarkers = retainDeletes) - // check if the batch contains the last sequence number for the producer. if so, we cannot - // remove the batch just yet or the producer may see an out of sequence error. - if (batch.hasProducerId && activeProducers.get(batch.producerId).contains(batch.lastSequence)) + def isBatchLastRecordOfProducer: Boolean = { + // We retain the batch in order to preserve the state of active producers. There are three cases: + // 1) The producer is no longer active, which means we can delete all records for that producer. + // 2) The producer is still active and has a last data offset. We retain the batch that contains + // this offset since it also contains the last sequence number for this producer. + // 3) The last entry in the log is a transaction marker. We retain this marker since it has the + // last producer epoch, which is needed to ensure fencing. + lastRecordsOfActiveProducers.get(batch.producerId).exists { lastRecord => + lastRecord.lastDataOffset match { + case Some(offset) => batch.lastOffset == offset + case None => batch.isControlBatch && batch.producerEpoch == lastRecord.producerEpoch + } + } + } + + if (batch.hasProducerId && isBatchLastRecordOfProducer) BatchRetention.RETAIN_EMPTY else if (discardBatchRecords) BatchRetention.DELETE @@ -607,7 +621,7 @@ private[log] class Cleaner(val id: Int, position += result.bytesRead // if any messages are to be retained, write them out - val outputBuffer = result.output + val outputBuffer = result.outputBuffer if (outputBuffer.position() > 0) { outputBuffer.flip() val retained = MemoryRecords.readableRecords(outputBuffer) @@ -980,24 +994,30 @@ private[log] class CleanedTransactionMetadata(val abortedTransactions: mutable.P def onControlBatchRead(controlBatch: RecordBatch): Boolean = { consumeAbortedTxnsUpTo(controlBatch.lastOffset) - val controlRecord = controlBatch.iterator.next() - val controlType = ControlRecordType.parse(controlRecord.key) - val producerId = controlBatch.producerId - controlType match { - case ControlRecordType.ABORT => - ongoingAbortedTxns.remove(producerId) match { - // Retain the marker until all batches from the transaction have been removed - case Some(abortedTxnMetadata) if abortedTxnMetadata.lastObservedBatchOffset.isDefined => - transactionIndex.foreach(_.append(abortedTxnMetadata.abortedTxn)) - false - case _ => true - } + val controlRecordIterator = controlBatch.iterator + if (controlRecordIterator.hasNext) { + val controlRecord = controlRecordIterator.next() + val controlType = ControlRecordType.parse(controlRecord.key) + val producerId = controlBatch.producerId + controlType match { + case ControlRecordType.ABORT => + ongoingAbortedTxns.remove(producerId) match { + // Retain the marker until all batches from the transaction have been removed + case Some(abortedTxnMetadata) if abortedTxnMetadata.lastObservedBatchOffset.isDefined => + transactionIndex.foreach(_.append(abortedTxnMetadata.abortedTxn)) + false + case _ => true + } - case ControlRecordType.COMMIT => - // This marker is eligible for deletion if we didn't traverse any batches from the transaction - !ongoingCommittedTxns.remove(producerId) + case ControlRecordType.COMMIT => + // This marker is eligible for deletion if we didn't traverse any batches from the transaction + !ongoingCommittedTxns.remove(producerId) - case _ => false + case _ => false + } + } else { + // An empty control batch was already cleaned, so it's safe to discard + true } } diff --git a/core/src/main/scala/kafka/log/LogConfig.scala b/core/src/main/scala/kafka/log/LogConfig.scala index 158209a1fc0a2..bd4768eda25fa 100755 --- a/core/src/main/scala/kafka/log/LogConfig.scala +++ b/core/src/main/scala/kafka/log/LogConfig.scala @@ -21,12 +21,12 @@ import java.util.{Collections, Locale, Properties} import scala.collection.JavaConverters._ import kafka.api.ApiVersion -import kafka.message.{BrokerCompressionCodec, Message} +import kafka.message.BrokerCompressionCodec import kafka.server.{KafkaConfig, ThrottledReplicaListValidator} import kafka.utils.Implicits._ import org.apache.kafka.common.errors.InvalidConfigurationException import org.apache.kafka.common.config.{AbstractConfig, ConfigDef, TopicConfig} -import org.apache.kafka.common.record.TimestampType +import org.apache.kafka.common.record.{LegacyRecord, TimestampType} import org.apache.kafka.common.utils.Utils import scala.collection.{Map, mutable} @@ -63,6 +63,7 @@ object Defaults { val LeaderReplicationThrottledReplicas = Collections.emptyList[String]() val FollowerReplicationThrottledReplicas = Collections.emptyList[String]() val MaxIdMapSnapshots = kafka.server.Defaults.MaxIdMapSnapshots + val MessageDownConversionEnable = kafka.server.Defaults.MessageDownConversionEnable } case class LogConfig(props: java.util.Map[_, _], overriddenConfigs: Set[String] = Set.empty) @@ -96,6 +97,7 @@ case class LogConfig(props: java.util.Map[_, _], overriddenConfigs: Set[String] val messageTimestampDifferenceMaxMs = getLong(LogConfig.MessageTimestampDifferenceMaxMsProp).longValue val LeaderReplicationThrottledReplicas = getList(LogConfig.LeaderReplicationThrottledReplicasProp) val FollowerReplicationThrottledReplicas = getList(LogConfig.FollowerReplicationThrottledReplicasProp) + val messageDownConversionEnable = getBoolean(LogConfig.MessageDownConversionEnableProp) def randomSegmentJitter: Long = if (segmentJitterMs == 0) 0 else Utils.abs(scala.util.Random.nextInt()) % math.min(segmentJitterMs, segmentMs) @@ -131,6 +133,7 @@ object LogConfig { val MessageFormatVersionProp = TopicConfig.MESSAGE_FORMAT_VERSION_CONFIG val MessageTimestampTypeProp = TopicConfig.MESSAGE_TIMESTAMP_TYPE_CONFIG val MessageTimestampDifferenceMaxMsProp = TopicConfig.MESSAGE_TIMESTAMP_DIFFERENCE_MAX_MS_CONFIG + val MessageDownConversionEnableProp = TopicConfig.MESSAGE_DOWNCONVERSION_ENABLE_CONFIG // Leave these out of TopicConfig for now as they are replication quota configs val LeaderReplicationThrottledReplicasProp = "leader.replication.throttled.replicas" @@ -158,6 +161,7 @@ object LogConfig { val MessageFormatVersionDoc = TopicConfig.MESSAGE_FORMAT_VERSION_DOC val MessageTimestampTypeDoc = TopicConfig.MESSAGE_TIMESTAMP_TYPE_DOC val MessageTimestampDifferenceMaxMsDoc = TopicConfig.MESSAGE_TIMESTAMP_DIFFERENCE_MAX_MS_DOC + val MessageDownConversionEnableDoc = TopicConfig.MESSAGE_DOWNCONVERSION_ENABLE_DOC val LeaderReplicationThrottledReplicasDoc = "A list of replicas for which log replication should be throttled on " + "the leader side. The list should describe a set of replicas in the form " + @@ -212,7 +216,7 @@ object LogConfig { import org.apache.kafka.common.config.ConfigDef.ValidString._ new LogConfigDef() - .define(SegmentBytesProp, INT, Defaults.SegmentSize, atLeast(Message.MinMessageOverhead), MEDIUM, + .define(SegmentBytesProp, INT, Defaults.SegmentSize, atLeast(LegacyRecord.RECORD_OVERHEAD_V0), MEDIUM, SegmentSizeDoc, KafkaConfig.LogSegmentBytesProp) .define(SegmentMsProp, LONG, Defaults.SegmentMs, atLeast(1), MEDIUM, SegmentMsDoc, KafkaConfig.LogRollTimeMillisProp) @@ -262,6 +266,8 @@ object LogConfig { LeaderReplicationThrottledReplicasDoc, LeaderReplicationThrottledReplicasProp) .define(FollowerReplicationThrottledReplicasProp, LIST, Defaults.FollowerReplicationThrottledReplicas, ThrottledReplicaListValidator, MEDIUM, FollowerReplicationThrottledReplicasDoc, FollowerReplicationThrottledReplicasProp) + .define(MessageDownConversionEnableProp, BOOLEAN, Defaults.MessageDownConversionEnable, LOW, + MessageDownConversionEnableDoc, KafkaConfig.LogMessageDownConversionEnableProp) } def apply(): LogConfig = LogConfig(new Properties()) @@ -325,7 +331,8 @@ object LogConfig { PreAllocateEnableProp -> KafkaConfig.LogPreAllocateProp, MessageFormatVersionProp -> KafkaConfig.LogMessageFormatVersionProp, MessageTimestampTypeProp -> KafkaConfig.LogMessageTimestampTypeProp, - MessageTimestampDifferenceMaxMsProp -> KafkaConfig.LogMessageTimestampDifferenceMaxMsProp + MessageTimestampDifferenceMaxMsProp -> KafkaConfig.LogMessageTimestampDifferenceMaxMsProp, + MessageDownConversionEnableProp -> KafkaConfig.LogMessageDownConversionEnableProp ) } diff --git a/core/src/main/scala/kafka/log/LogManager.scala b/core/src/main/scala/kafka/log/LogManager.scala index c0ac3b819071d..b4b0a57d2b74b 100755 --- a/core/src/main/scala/kafka/log/LogManager.scala +++ b/core/src/main/scala/kafka/log/LogManager.scala @@ -22,13 +22,12 @@ import java.nio.file.Files import java.util.concurrent._ import com.yammer.metrics.core.Gauge -import kafka.common.KafkaException import kafka.metrics.KafkaMetricsGroup import kafka.server.checkpoints.OffsetCheckpointFile import kafka.server.{BrokerState, RecoveringFromUncleanShutdown, _} import kafka.utils._ import kafka.zk.KafkaZkClient -import org.apache.kafka.common.TopicPartition +import org.apache.kafka.common.{KafkaException, TopicPartition} import org.apache.kafka.common.utils.Time import org.apache.kafka.common.errors.{KafkaStorageException, LogDirNotFoundException} @@ -144,10 +143,8 @@ class LogManager(logDirs: Seq[File], * */ private def createAndValidateLogDirs(dirs: Seq[File], initialOfflineDirs: Seq[File]): ConcurrentLinkedQueue[File] = { - if(dirs.map(_.getCanonicalPath).toSet.size < dirs.size) - throw new KafkaException("Duplicate log directory found: " + dirs.mkString(", ")) - val liveLogDirs = new ConcurrentLinkedQueue[File]() + val canonicalPaths = mutable.HashSet.empty[String] for (dir <- dirs) { try { @@ -155,13 +152,21 @@ class LogManager(logDirs: Seq[File], throw new IOException(s"Failed to load ${dir.getAbsolutePath} during broker startup") if (!dir.exists) { - info("Log directory '" + dir.getAbsolutePath + "' not found, creating it.") + info(s"Log directory ${dir.getAbsolutePath} not found, creating it.") val created = dir.mkdirs() if (!created) - throw new IOException("Failed to create data directory " + dir.getAbsolutePath) + throw new IOException(s"Failed to create data directory ${dir.getAbsolutePath}") } if (!dir.isDirectory || !dir.canRead) - throw new IOException(dir.getAbsolutePath + " is not a readable log directory.") + throw new IOException(s"${dir.getAbsolutePath} is not a readable log directory.") + + // getCanonicalPath() throws IOException if a file system query fails or if the path is invalid (e.g. contains + // the Nul character). Since there's no easy way to distinguish between the two cases, we treat them the same + // and mark the log directory as offline. + if (!canonicalPaths.add(dir.getCanonicalPath)) + throw new KafkaException(s"Duplicate log directory found: ${dirs.mkString(", ")}") + + liveLogDirs.add(dir) } catch { case e: IOException => @@ -169,7 +174,7 @@ class LogManager(logDirs: Seq[File], } } if (liveLogDirs.isEmpty) { - fatal(s"Shutdown broker because none of the specified log dirs from " + dirs.mkString(", ") + " can be created or validated") + fatal(s"Shutdown broker because none of the specified log dirs from ${dirs.mkString(", ")} can be created or validated") Exit.halt(1) } @@ -831,7 +836,7 @@ class LogManager(logDirs: Seq[File], addLogToBeDeleted(removedLog) info(s"Log for partition ${removedLog.topicPartition} is renamed to ${removedLog.dir.getAbsolutePath} and is scheduled for deletion") } else if (offlineLogDirs.nonEmpty) { - throw new KafkaStorageException("Failed to delete log for " + topicPartition + " because it may be in one of the offline directories " + offlineLogDirs.mkString(",")) + throw new KafkaStorageException(s"Failed to delete log for ${if (isFuture) "future" else ""} $topicPartition because it may be in one of the offline directories ${offlineLogDirs.mkString(",")}") } removedLog } diff --git a/core/src/main/scala/kafka/log/LogSegment.scala b/core/src/main/scala/kafka/log/LogSegment.scala index 55ab088f55e64..85dd61afc91ac 100755 --- a/core/src/main/scala/kafka/log/LogSegment.scala +++ b/core/src/main/scala/kafka/log/LogSegment.scala @@ -21,11 +21,12 @@ import java.nio.file.{Files, NoSuchFileException} import java.nio.file.attribute.FileTime import java.util.concurrent.TimeUnit +import kafka.common.LogSegmentOffsetOverflowException import kafka.metrics.{KafkaMetricsGroup, KafkaTimer} -import kafka.server.epoch.LeaderEpochCache +import kafka.server.epoch.LeaderEpochFileCache import kafka.server.{FetchDataInfo, LogOffsetMetadata} import kafka.utils._ -import org.apache.kafka.common.errors.CorruptRecordException +import org.apache.kafka.common.errors.{CorruptRecordException, InvalidOffsetException} import org.apache.kafka.common.record.FileRecords.LogOffsetPosition import org.apache.kafka.common.record._ import org.apache.kafka.common.utils.Time @@ -103,7 +104,7 @@ class LogSegment private[log] (val log: FileRecords, * checks that the argument offset can be represented as an integer offset relative to the baseOffset. */ def canConvertToRelativeOffset(offset: Long): Boolean = { - (offset - baseOffset) <= Integer.MAX_VALUE + offsetIndex.canAppendOffset(offset) } /** @@ -117,6 +118,7 @@ class LogSegment private[log] (val log: FileRecords, * @param shallowOffsetOfMaxTimestamp The offset of the message that has the largest timestamp in the messages to append. * @param records The log entries to append. * @return the physical position in the file of the appended records + * @throws LogSegmentOffsetOverflowException if the largest offset causes index offset overflow */ @nonthreadsafe def append(largestOffset: Long, @@ -129,17 +131,19 @@ class LogSegment private[log] (val log: FileRecords, val physicalPosition = log.sizeInBytes() if (physicalPosition == 0) rollingBasedTimestamp = Some(largestTimestamp) + + ensureOffsetInRange(largestOffset) + // append the messages - require(canConvertToRelativeOffset(largestOffset), "largest offset in message set can not be safely converted to relative offset.") val appendedBytes = log.append(records) - trace(s"Appended $appendedBytes to ${log.file()} at end offset $largestOffset") + trace(s"Appended $appendedBytes to ${log.file} at end offset $largestOffset") // Update the in memory max timestamp and corresponding offset. if (largestTimestamp > maxTimestampSoFar) { maxTimestampSoFar = largestTimestamp offsetOfMaxTimestamp = shallowOffsetOfMaxTimestamp } // append an entry to the index (if needed) - if(bytesSinceLastIndexEntry > indexIntervalBytes) { + if (bytesSinceLastIndexEntry > indexIntervalBytes) { offsetIndex.append(largestOffset, physicalPosition) timeIndex.maybeAppend(maxTimestampSoFar, offsetOfMaxTimestamp) bytesSinceLastIndexEntry = 0 @@ -148,6 +152,68 @@ class LogSegment private[log] (val log: FileRecords, } } + private def ensureOffsetInRange(offset: Long): Unit = { + if (!canConvertToRelativeOffset(offset)) + throw new LogSegmentOffsetOverflowException(this, offset) + } + + private def appendChunkFromFile(records: FileRecords, position: Int, bufferSupplier: BufferSupplier): Int = { + var bytesToAppend = 0 + var maxTimestamp = Long.MinValue + var offsetOfMaxTimestamp = Long.MinValue + var maxOffset = Long.MinValue + var readBuffer = bufferSupplier.get(1024 * 1024) + + def canAppend(batch: RecordBatch) = + canConvertToRelativeOffset(batch.lastOffset) && + (bytesToAppend == 0 || bytesToAppend + batch.sizeInBytes < readBuffer.capacity) + + // find all batches that are valid to be appended to the current log segment and + // determine the maximum offset and timestamp + val nextBatches = records.batchesFrom(position).asScala.iterator + for (batch <- nextBatches.takeWhile(canAppend)) { + if (batch.maxTimestamp > maxTimestamp) { + maxTimestamp = batch.maxTimestamp + offsetOfMaxTimestamp = batch.lastOffset + } + maxOffset = batch.lastOffset + bytesToAppend += batch.sizeInBytes + } + + if (bytesToAppend > 0) { + // Grow buffer if needed to ensure we copy at least one batch + if (readBuffer.capacity < bytesToAppend) + readBuffer = bufferSupplier.get(bytesToAppend) + + readBuffer.limit(bytesToAppend) + records.readInto(readBuffer, position) + + append(maxOffset, maxTimestamp, offsetOfMaxTimestamp, MemoryRecords.readableRecords(readBuffer)) + } + + bufferSupplier.release(readBuffer) + bytesToAppend + } + + /** + * Append records from a file beginning at the given position until either the end of the file + * is reached or an offset is found which is too large to convert to a relative offset for the indexes. + * + * @return the number of bytes appended to the log (may be less than the size of the input if an + * offset is encountered which would overflow this segment) + */ + def appendFromFile(records: FileRecords, start: Int): Int = { + var position = start + val bufferSupplier: BufferSupplier = new BufferSupplier.GrowableBufferSupplier + while (position < start + records.sizeInBytes) { + val bytesAppended = appendChunkFromFile(records, position, bufferSupplier) + if (bytesAppended == 0) + return position - start + position += bytesAppended + } + position - start + } + @nonthreadsafe def updateTxnIndex(completedTxn: CompletedTxn, lastStableOffset: Long) { if (completedTxn.isAborted) { @@ -163,8 +229,9 @@ class LogSegment private[log] (val log: FileRecords, val maybeCompletedTxn = appendInfo.append(batch) producerStateManager.update(appendInfo) maybeCompletedTxn.foreach { completedTxn => - val lastStableOffset = producerStateManager.completeTxn(completedTxn) + val lastStableOffset = producerStateManager.lastStableOffset(completedTxn) updateTxnIndex(completedTxn, lastStableOffset) + producerStateManager.completeTxn(completedTxn) } } producerStateManager.updateMapEndOffset(batch.lastOffset + 1) @@ -193,8 +260,8 @@ class LogSegment private[log] (val log: FileRecords, * no more than maxSize bytes and will end before maxOffset if a maxOffset is specified. * * @param startOffset A lower bound on the first offset to include in the message set we read - * @param maxSize The maximum number of bytes to include in the message set we read * @param maxOffset An optional maximum offset for the message set we read + * @param maxSize The maximum number of bytes to include in the message set we read * @param maxPosition The maximum position in the log segment that should be exposed for read * @param minOneMessage If this is true, the first message will be returned even if it exceeds `maxSize` (if one exists) * @@ -246,7 +313,7 @@ class LogSegment private[log] (val log: FileRecords, min(min(maxPosition, endPosition) - startPosition, adjustedMaxSize).toInt } - FetchDataInfo(offsetMetadata, log.read(startPosition, fetchSize), + FetchDataInfo(offsetMetadata, log.slice(startPosition, fetchSize), firstEntryIncomplete = adjustedMaxSize < startOffsetAndSize.size) } @@ -261,9 +328,10 @@ class LogSegment private[log] (val log: FileRecords, * the transaction index. * @param leaderEpochCache Optionally a cache for updating the leader epoch during recovery. * @return The number of bytes truncated from the log + * @throws LogSegmentOffsetOverflowException if the log segment contains an offset that causes the index offset to overflow */ @nonthreadsafe - def recover(producerStateManager: ProducerStateManager, leaderEpochCache: Option[LeaderEpochCache] = None): Int = { + def recover(producerStateManager: ProducerStateManager, leaderEpochCache: Option[LeaderEpochFileCache] = None): Int = { offsetIndex.reset() timeIndex.reset() txnIndex.reset() @@ -273,6 +341,7 @@ class LogSegment private[log] (val log: FileRecords, try { for (batch <- log.batches.asScala) { batch.ensureValid() + ensureOffsetInRange(batch.lastOffset) // The max timestamp is exposed at the batch level, so no need to iterate the records if (batch.maxTimestamp > maxTimestampSoFar) { @@ -282,8 +351,7 @@ class LogSegment private[log] (val log: FileRecords, // Build offset index if (validBytes - lastIndexEntry > indexIntervalBytes) { - val startOffset = batch.baseOffset - offsetIndex.append(startOffset, validBytes) + offsetIndex.append(batch.lastOffset, validBytes) timeIndex.maybeAppend(maxTimestampSoFar, offsetOfMaxTimestamp) lastIndexEntry = validBytes } @@ -291,7 +359,7 @@ class LogSegment private[log] (val log: FileRecords, if (batch.magic >= RecordBatch.MAGIC_VALUE_V2) { leaderEpochCache.foreach { cache => - if (batch.partitionLeaderEpoch > cache.latestEpoch()) // this is to avoid unnecessary warning in cache.assign() + if (batch.partitionLeaderEpoch > cache.latestEpoch) // this is to avoid unnecessary warning in cache.assign() cache.assign(batch.partitionLeaderEpoch, batch.baseOffset) } updateProducerState(producerStateManager, batch) @@ -329,6 +397,14 @@ class LogSegment private[log] (val log: FileRecords, } } + /** + * Check whether the last offset of the last batch in this segment overflows the indexes. + */ + def hasOverflow: Boolean = { + val nextOffset = readNextOffset + nextOffset > baseOffset && !canConvertToRelativeOffset(nextOffset - 1) + } + def collectAbortedTxns(fetchOffset: Long, upperBoundOffset: Long): TxnIndexSearchResult = txnIndex.collectAbortedTxns(fetchOffset, upperBoundOffset) @@ -372,11 +448,11 @@ class LogSegment private[log] (val log: FileRecords, */ @threadsafe def readNextOffset: Long = { - val ms = read(offsetIndex.lastOffset, None, log.sizeInBytes) - if (ms == null) + val fetchData = read(offsetIndex.lastOffset, None, log.sizeInBytes) + if (fetchData == null) baseOffset else - ms.records.batches.asScala.lastOption + fetchData.records.batches.asScala.lastOption .map(_.nextOffset) .getOrElse(baseOffset) } @@ -546,6 +622,7 @@ class LogSegment private[log] (val log: FileRecords, Files.setLastModifiedTime(offsetIndex.file.toPath, fileTime) Files.setLastModifiedTime(timeIndex.file.toPath, fileTime) } + } object LogSegment { @@ -566,6 +643,12 @@ object LogSegment { time) } + def deleteIfExists(dir: File, baseOffset: Long, fileSuffix: String = ""): Unit = { + Log.deleteFileIfExists(Log.offsetIndexFile(dir, baseOffset, fileSuffix)) + Log.deleteFileIfExists(Log.timeIndexFile(dir, baseOffset, fileSuffix)) + Log.deleteFileIfExists(Log.transactionIndexFile(dir, baseOffset, fileSuffix)) + Log.deleteFileIfExists(Log.logFile(dir, baseOffset, fileSuffix)) + } } object LogFlushStats extends KafkaMetricsGroup { diff --git a/core/src/main/scala/kafka/log/OffsetIndex.scala b/core/src/main/scala/kafka/log/OffsetIndex.scala index 523c88c772345..4da3dde87a5ff 100755 --- a/core/src/main/scala/kafka/log/OffsetIndex.scala +++ b/core/src/main/scala/kafka/log/OffsetIndex.scala @@ -21,7 +21,7 @@ import java.io.File import java.nio.ByteBuffer import kafka.utils.CoreUtils.inLock -import kafka.common.InvalidOffsetException +import org.apache.kafka.common.errors.InvalidOffsetException /** * An index that maps offsets to physical file locations for a particular log segment. This index may be sparse: @@ -57,8 +57,8 @@ class OffsetIndex(_file: File, baseOffset: Long, maxIndexSize: Int = -1, writabl /* the last offset in the index */ private[this] var _lastOffset = lastEntry.offset - debug("Loaded index file %s with maxEntries = %d, maxIndexSize = %d, entries = %d, lastOffset = %d, file position = %d" - .format(file.getAbsolutePath, maxEntries, maxIndexSize, _entries, _lastOffset, mmap.position())) + debug(s"Loaded index file ${file.getAbsolutePath} with maxEntries = $maxEntries, " + + s"maxIndexSize = $maxIndexSize, entries = ${_entries}, lastOffset = ${_lastOffset}, file position = ${mmap.position()}") /** * The last entry in the index @@ -126,7 +126,8 @@ class OffsetIndex(_file: File, baseOffset: Long, maxIndexSize: Int = -1, writabl def entry(n: Int): OffsetPosition = { maybeLock(lock) { if(n >= _entries) - throw new IllegalArgumentException("Attempt to fetch the %dth entry from an index of size %d.".format(n, _entries)) + throw new IllegalArgumentException(s"Attempt to fetch the ${n}th entry from index ${file.getAbsolutePath}, " + + s"which has size ${_entries}.") val idx = mmap.duplicate OffsetPosition(relativeOffset(idx, n), physical(idx, n)) } @@ -134,20 +135,21 @@ class OffsetIndex(_file: File, baseOffset: Long, maxIndexSize: Int = -1, writabl /** * Append an entry for the given offset/location pair to the index. This entry must have a larger offset than all subsequent entries. + * @throws InvalidOffsetException if the offset causes index offset to overflow */ def append(offset: Long, position: Int) { inLock(lock) { require(!isFull, "Attempt to append to a full index (size = " + _entries + ").") if (_entries == 0 || offset > _lastOffset) { - debug("Adding index entry %d => %d to %s.".format(offset, position, file.getName)) - mmap.putInt((offset - baseOffset).toInt) + trace(s"Adding index entry $offset => $position to ${file.getAbsolutePath}") + mmap.putInt(relativeOffset(offset)) mmap.putInt(position) _entries += 1 _lastOffset = offset require(_entries * entrySize == mmap.position(), entries + " entries but file position in index is " + mmap.position() + ".") } else { - throw new InvalidOffsetException("Attempt to append an offset (%d) to position %d no larger than the last offset appended (%d) to %s." - .format(offset, entries, _lastOffset, file.getAbsolutePath)) + throw new InvalidOffsetException(s"Attempt to append an offset ($offset) to position $entries no larger than" + + s" the last offset appended (${_lastOffset}) to ${file.getAbsolutePath}.") } } } @@ -183,13 +185,15 @@ class OffsetIndex(_file: File, baseOffset: Long, maxIndexSize: Int = -1, writabl _entries = entries mmap.position(_entries * entrySize) _lastOffset = lastEntry.offset + debug(s"Truncated index ${file.getAbsolutePath} to $entries entries;" + + s" position is now ${mmap.position()} and last offset is now ${_lastOffset}") } } override def sanityCheck() { - if (_entries != 0 && _lastOffset <= baseOffset) + if (_entries != 0 && _lastOffset < baseOffset) throw new CorruptIndexException(s"Corrupt index found, index file (${file.getAbsolutePath}) has non-zero size " + - s"but the last offset is ${_lastOffset} which is no greater than the base offset $baseOffset.") + s"but the last offset is ${_lastOffset} which is less than the base offset $baseOffset.") if (length % entrySize != 0) throw new CorruptIndexException(s"Index file ${file.getAbsolutePath} is corrupt, found $length bytes which is " + s"neither positive nor a multiple of $entrySize.") diff --git a/core/src/main/scala/kafka/log/ProducerStateManager.scala b/core/src/main/scala/kafka/log/ProducerStateManager.scala index abeac6e6618f4..cd51032c1e622 100644 --- a/core/src/main/scala/kafka/log/ProducerStateManager.scala +++ b/core/src/main/scala/kafka/log/ProducerStateManager.scala @@ -20,22 +20,28 @@ import java.io._ import java.nio.ByteBuffer import java.nio.file.Files -import kafka.common.KafkaException import kafka.log.Log.offsetFromFile import kafka.server.LogOffsetMetadata import kafka.utils.{Logging, nonthreadsafe, threadsafe} -import org.apache.kafka.common.TopicPartition +import org.apache.kafka.common.{KafkaException, TopicPartition} import org.apache.kafka.common.errors._ import org.apache.kafka.common.internals.Topic import org.apache.kafka.common.protocol.types._ -import org.apache.kafka.common.record.{ControlRecordType, EndTransactionMarker, RecordBatch} +import org.apache.kafka.common.record.{ControlRecordType, DefaultRecordBatch, EndTransactionMarker, RecordBatch} import org.apache.kafka.common.utils.{ByteUtils, Crc32C} +import scala.collection.JavaConverters._ import scala.collection.mutable.ListBuffer import scala.collection.{immutable, mutable} class CorruptSnapshotException(msg: String) extends KafkaException(msg) +/** + * The last written record for a given producer. The last data offset may be undefined + * if the only log entry for a producer is a transaction marker. + */ +case class LastRecord(lastDataOffset: Option[Long], producerEpoch: Short) + // ValidationType and its subtypes define the extent of the validation to perform on a given ProducerAppendInfo instance private[log] sealed trait ValidationType @@ -76,7 +82,7 @@ private[log] object ProducerStateEntry { } private[log] case class BatchMetadata(lastSeq: Int, lastOffset: Long, offsetDelta: Int, timestamp: Long) { - def firstSeq = lastSeq - offsetDelta + def firstSeq = DefaultRecordBatch.decrementSequence(lastSeq, offsetDelta) def firstOffset = lastOffset - offsetDelta override def toString: String = { @@ -184,7 +190,8 @@ private[log] class ProducerStateEntry(val producerId: Long, * should have ValidationType.None. Appends coming from a client for produce requests should have * ValidationType.Full. */ -private[log] class ProducerAppendInfo(val producerId: Long, +private[log] class ProducerAppendInfo(val topicPartition: TopicPartition, + val producerId: Long, val currentEntry: ProducerStateEntry, val validationType: ValidationType) { private val transactions = ListBuffer.empty[TxnMetadata] @@ -194,35 +201,36 @@ private[log] class ProducerAppendInfo(val producerId: Long, updatedEntry.coordinatorEpoch = currentEntry.coordinatorEpoch updatedEntry.currentTxnFirstOffset = currentEntry.currentTxnFirstOffset - private def maybeValidateAppend(producerEpoch: Short, firstSeq: Int) = { + private def maybeValidateAppend(producerEpoch: Short, firstSeq: Int, offset: Long): Unit = { validationType match { case ValidationType.None => case ValidationType.EpochOnly => - checkProducerEpoch(producerEpoch) + checkProducerEpoch(producerEpoch, offset) case ValidationType.Full => - checkProducerEpoch(producerEpoch) - checkSequence(producerEpoch, firstSeq) + checkProducerEpoch(producerEpoch, offset) + checkSequence(producerEpoch, firstSeq, offset) } } - private def checkProducerEpoch(producerEpoch: Short): Unit = { + private def checkProducerEpoch(producerEpoch: Short, offset: Long): Unit = { if (producerEpoch < updatedEntry.producerEpoch) { - throw new ProducerFencedException(s"Producer's epoch is no longer valid. There is probably another producer " + - s"with a newer epoch. $producerEpoch (request epoch), ${updatedEntry.producerEpoch} (server epoch)") + throw new ProducerFencedException(s"Producer's epoch at offset $offset is no longer valid in " + + s"partition $topicPartition: $producerEpoch (request epoch), ${updatedEntry.producerEpoch} (current epoch)") } } - private def checkSequence(producerEpoch: Short, appendFirstSeq: Int): Unit = { + private def checkSequence(producerEpoch: Short, appendFirstSeq: Int, offset: Long): Unit = { if (producerEpoch != updatedEntry.producerEpoch) { if (appendFirstSeq != 0) { if (updatedEntry.producerEpoch != RecordBatch.NO_PRODUCER_EPOCH) { - throw new OutOfOrderSequenceException(s"Invalid sequence number for new epoch: $producerEpoch " + - s"(request epoch), $appendFirstSeq (seq. number)") + throw new OutOfOrderSequenceException(s"Invalid sequence number for new epoch at offset $offset in " + + s"partition $topicPartition: $producerEpoch (request epoch), $appendFirstSeq (seq. number)") } else { - throw new UnknownProducerIdException(s"Found no record of producerId=$producerId on the broker. It is possible " + - s"that the last message with the producerId=$producerId has been removed due to hitting the retention limit.") + throw new UnknownProducerIdException(s"Found no record of producerId=$producerId on the broker at offset $offset" + + s"in partition $topicPartition. It is possible that the last message with the producerId=$producerId has " + + "been removed due to hitting the retention limit.") } } } else { @@ -234,12 +242,18 @@ private[log] class ProducerAppendInfo(val producerId: Long, RecordBatch.NO_SEQUENCE if (currentLastSeq == RecordBatch.NO_SEQUENCE && appendFirstSeq != 0) { - // the epoch was bumped by a control record, so we expect the sequence number to be reset - throw new OutOfOrderSequenceException(s"Out of order sequence number for producerId $producerId: found $appendFirstSeq " + - s"(incoming seq. number), but expected 0") + // We have a matching epoch, but we do not know the next sequence number. This case can happen if + // only a transaction marker is left in the log for this producer. We treat this as an unknown + // producer id error, so that the producer can check the log start offset for truncation and reset + // the sequence number. Note that this check follows the fencing check, so the marker still fences + // old producers even if it cannot determine our next expected sequence number. + throw new UnknownProducerIdException(s"Local producer state matches expected epoch $producerEpoch " + + s"for producerId=$producerId at offset $offset in partition $topicPartition, but the next expected " + + "sequence number is not known.") } else if (!inSequence(currentLastSeq, appendFirstSeq)) { - throw new OutOfOrderSequenceException(s"Out of order sequence number for producerId $producerId: $appendFirstSeq " + - s"(incoming seq. number), $currentLastSeq (current end sequence number)") + throw new OutOfOrderSequenceException(s"Out of order sequence number for producerId $producerId at " + + s"offset $offset in partition $topicPartition: $appendFirstSeq (incoming seq. number), " + + s"$currentLastSeq (current end sequence number)") } } } @@ -250,12 +264,18 @@ private[log] class ProducerAppendInfo(val producerId: Long, def append(batch: RecordBatch): Option[CompletedTxn] = { if (batch.isControlBatch) { - val record = batch.iterator.next() - val endTxnMarker = EndTransactionMarker.deserialize(record) - val completedTxn = appendEndTxnMarker(endTxnMarker, batch.producerEpoch, batch.baseOffset, record.timestamp) - Some(completedTxn) + val recordIterator = batch.iterator + if (recordIterator.hasNext) { + val record = recordIterator.next() + val endTxnMarker = EndTransactionMarker.deserialize(record) + val completedTxn = appendEndTxnMarker(endTxnMarker, batch.producerEpoch, batch.baseOffset, record.timestamp) + Some(completedTxn) + } else { + // An empty control batch means the entire transaction has been cleaned from the log, so no need to append + None + } } else { - append(batch.producerEpoch, batch.baseSequence, batch.lastSequence, batch.maxTimestamp, batch.lastOffset, + append(batch.producerEpoch, batch.baseSequence, batch.lastSequence, batch.maxTimestamp, batch.baseOffset, batch.lastOffset, batch.isTransactional) None } @@ -265,19 +285,20 @@ private[log] class ProducerAppendInfo(val producerId: Long, firstSeq: Int, lastSeq: Int, lastTimestamp: Long, + firstOffset: Long, lastOffset: Long, isTransactional: Boolean): Unit = { - maybeValidateAppend(epoch, firstSeq) - updatedEntry.addBatch(epoch, lastSeq, lastOffset, lastSeq - firstSeq, lastTimestamp) + maybeValidateAppend(epoch, firstSeq, firstOffset) + updatedEntry.addBatch(epoch, lastSeq, lastOffset, (lastOffset - firstOffset).toInt, lastTimestamp) updatedEntry.currentTxnFirstOffset match { case Some(_) if !isTransactional => // Received a non-transactional message while a transaction is active - throw new InvalidTxnStateException(s"Expected transactional write from producer $producerId") + throw new InvalidTxnStateException(s"Expected transactional write from producer $producerId at " + + s"offset $firstOffset in partition $topicPartition") case None if isTransactional => // Began a new transaction - val firstOffset = lastOffset - (lastSeq - firstSeq) updatedEntry.currentTxnFirstOffset = Some(firstOffset) transactions += new TxnMetadata(producerId, firstOffset) @@ -289,10 +310,11 @@ private[log] class ProducerAppendInfo(val producerId: Long, producerEpoch: Short, offset: Long, timestamp: Long): CompletedTxn = { - checkProducerEpoch(producerEpoch) + checkProducerEpoch(producerEpoch, offset) if (updatedEntry.coordinatorEpoch > endTxnMarker.coordinatorEpoch) - throw new TransactionCoordinatorFencedException(s"Invalid coordinator epoch: ${endTxnMarker.coordinatorEpoch} " + + throw new TransactionCoordinatorFencedException(s"Invalid coordinator epoch for producerId $producerId at " + + s"offset $offset in partition $topicPartition: ${endTxnMarker.coordinatorEpoch} " + s"(zombie), ${updatedEntry.coordinatorEpoch} (current)") updatedEntry.maybeUpdateEpoch(producerEpoch) @@ -592,7 +614,7 @@ class ProducerStateManager(val topicPartition: TopicPartition, * snapshot in range (if there is one). Note that the log end offset is assumed to be less than * or equal to the high watermark. */ - def truncateAndReload(logStartOffset: Long, logEndOffset: Long, currentTimeMs: Long) { + def truncateAndReload(logStartOffset: Long, logEndOffset: Long, currentTimeMs: Long): Unit = { // remove all out of range snapshots deleteSnapshotFiles(logDir, { snapOffset => snapOffset > logEndOffset || snapOffset <= logStartOffset @@ -621,7 +643,7 @@ class ProducerStateManager(val topicPartition: TopicPartition, ValidationType.Full val currentEntry = lastEntry(producerId).getOrElse(ProducerStateEntry.empty(producerId)) - new ProducerAppendInfo(producerId, currentEntry, validationToPerform) + new ProducerAppendInfo(topicPartition, producerId, currentEntry, validationToPerform) } /** @@ -744,9 +766,20 @@ class ProducerStateManager(val topicPartition: TopicPartition, } /** - * Complete the transaction and return the last stable offset. + * Compute the last stable offset of a completed transaction, but do not yet mark the transaction complete. + * That will be done in `completeTxn` below. This is used to compute the LSO that will be appended to the + * transaction index, but the completion must be done only after successfully appending to the index. */ - def completeTxn(completedTxn: CompletedTxn): Long = { + def lastStableOffset(completedTxn: CompletedTxn): Long = { + val nextIncompleteTxn = ongoingTxns.values.asScala.find(_.producerId != completedTxn.producerId) + nextIncompleteTxn.map(_.firstOffset.messageOffset).getOrElse(completedTxn.lastOffset + 1) + } + + /** + * Mark a transaction as completed. We will still await advancement of the high watermark before + * advancing the first unstable offset. + */ + def completeTxn(completedTxn: CompletedTxn): Unit = { val txnMetadata = ongoingTxns.remove(completedTxn.firstOffset) if (txnMetadata == null) throw new IllegalArgumentException(s"Attempted to complete transaction $completedTxn on partition $topicPartition " + @@ -754,9 +787,6 @@ class ProducerStateManager(val topicPartition: TopicPartition, txnMetadata.lastOffset = Some(completedTxn.lastOffset) unreplicatedTxns.put(completedTxn.firstOffset, txnMetadata) - - val lastStableOffset = firstUndecidedOffset.getOrElse(completedTxn.lastOffset + 1) - lastStableOffset } @threadsafe diff --git a/core/src/main/scala/kafka/log/TimeIndex.scala b/core/src/main/scala/kafka/log/TimeIndex.scala index e505f36aec390..6106ed2aed83d 100644 --- a/core/src/main/scala/kafka/log/TimeIndex.scala +++ b/core/src/main/scala/kafka/log/TimeIndex.scala @@ -20,9 +20,9 @@ package kafka.log import java.io.File import java.nio.ByteBuffer -import kafka.common.InvalidOffsetException import kafka.utils.CoreUtils._ import kafka.utils.Logging +import org.apache.kafka.common.errors.InvalidOffsetException import org.apache.kafka.common.record.RecordBatch /** @@ -57,6 +57,9 @@ class TimeIndex(_file: File, baseOffset: Long, maxIndexSize: Int = -1, writable: override def entrySize = 12 + debug(s"Loaded index file ${file.getAbsolutePath} with maxEntries = $maxEntries, maxIndexSize = $maxIndexSize," + + s" entries = ${_entries}, lastOffset = ${_lastEntry}, file position = ${mmap.position()}") + // We override the full check to reserve the last time index entry slot for the on roll call. override def isFull: Boolean = entries >= maxEntries - 1 @@ -86,7 +89,8 @@ class TimeIndex(_file: File, baseOffset: Long, maxIndexSize: Int = -1, writable: def entry(n: Int): TimestampOffset = { maybeLock(lock) { if(n >= _entries) - throw new IllegalArgumentException("Attempt to fetch the %dth entry from a time index of size %d.".format(n, _entries)) + throw new IllegalArgumentException(s"Attempt to fetch the ${n}th entry from time index ${file.getAbsolutePath} " + + s"which has size ${_entries}.") val idx = mmap.duplicate TimestampOffset(timestamp(idx, n), relativeOffset(idx, n)) } @@ -117,18 +121,18 @@ class TimeIndex(_file: File, baseOffset: Long, maxIndexSize: Int = -1, writable: // 1. A log segment is closed. // 2. LogSegment.onBecomeInactiveSegment() is called when an active log segment is rolled. if (_entries != 0 && offset < lastEntry.offset) - throw new InvalidOffsetException("Attempt to append an offset (%d) to slot %d no larger than the last offset appended (%d) to %s." - .format(offset, _entries, lastEntry.offset, file.getAbsolutePath)) + throw new InvalidOffsetException(s"Attempt to append an offset ($offset) to slot ${_entries} no larger than" + + s" the last offset appended (${lastEntry.offset}) to ${file.getAbsolutePath}.") if (_entries != 0 && timestamp < lastEntry.timestamp) - throw new IllegalStateException("Attempt to append a timestamp (%d) to slot %d no larger than the last timestamp appended (%d) to %s." - .format(timestamp, _entries, lastEntry.timestamp, file.getAbsolutePath)) + throw new IllegalStateException(s"Attempt to append a timestamp ($timestamp) to slot ${_entries} no larger" + + s" than the last timestamp appended (${lastEntry.timestamp}) to ${file.getAbsolutePath}.") // We only append to the time index when the timestamp is greater than the last inserted timestamp. // If all the messages are in message format v0, the timestamp will always be NoTimestamp. In that case, the time // index will be empty. if (timestamp > lastEntry.timestamp) { - debug("Adding index entry %d => %d to %s.".format(timestamp, offset, file.getName)) + trace(s"Adding index entry $timestamp => $offset to ${file.getAbsolutePath}.") mmap.putLong(timestamp) - mmap.putInt((offset - baseOffset).toInt) + mmap.putInt(relativeOffset(offset)) _entries += 1 _lastEntry = TimestampOffset(timestamp, offset) require(_entries * entrySize == mmap.position(), _entries + " entries but file position in index is " + mmap.position() + ".") @@ -202,6 +206,7 @@ class TimeIndex(_file: File, baseOffset: Long, maxIndexSize: Int = -1, writable: _entries = entries mmap.position(_entries * entrySize) _lastEntry = lastEntryFromIndexFile + debug(s"Truncated index ${file.getAbsolutePath} to $entries entries; position is now ${mmap.position()} and last entry is now ${_lastEntry}") } } diff --git a/core/src/main/scala/kafka/log/TransactionIndex.scala b/core/src/main/scala/kafka/log/TransactionIndex.scala index 349f0ce94ba58..160fb72ea6014 100644 --- a/core/src/main/scala/kafka/log/TransactionIndex.scala +++ b/core/src/main/scala/kafka/log/TransactionIndex.scala @@ -53,10 +53,11 @@ class TransactionIndex(val startOffset: Long, @volatile var file: File) extends def append(abortedTxn: AbortedTxn): Unit = { lastOffset.foreach { offset => if (offset >= abortedTxn.lastOffset) - throw new IllegalArgumentException("The last offset of appended transactions must increase sequentially") + throw new IllegalArgumentException(s"The last offset of appended transactions must increase sequentially, but " + + s"${abortedTxn.lastOffset} is not greater than current last offset $offset of index ${file.getAbsolutePath}") } lastOffset = Some(abortedTxn.lastOffset) - Utils.writeFully(channel, abortedTxn.buffer.duplicate()) + Utils.writeFully(channel(), abortedTxn.buffer.duplicate()) } def flush(): Unit = maybeChannel.foreach(_.force(true)) @@ -73,7 +74,7 @@ class TransactionIndex(val startOffset: Long, @volatile var file: File) extends Files.deleteIfExists(file.toPath) } - private def channel: FileChannel = { + private def channel(): FileChannel = { maybeChannel match { case Some(channel) => channel case None => openChannel() @@ -113,7 +114,7 @@ class TransactionIndex(val startOffset: Long, @volatile var file: File) extends var newLastOffset: Option[Long] = None for ((abortedTxn, position) <- iterator(() => buffer)) { if (abortedTxn.lastOffset >= offset) { - channel.truncate(position) + channel().truncate(position) lastOffset = newLastOffset return } @@ -138,8 +139,8 @@ class TransactionIndex(val startOffset: Long, @volatile var file: File) extends val abortedTxn = new AbortedTxn(buffer) if (abortedTxn.version > AbortedTxn.CurrentVersion) - throw new KafkaException(s"Unexpected aborted transaction version ${abortedTxn.version}, " + - s"current version is ${AbortedTxn.CurrentVersion}") + throw new KafkaException(s"Unexpected aborted transaction version ${abortedTxn.version} " + + s"in transaction index ${file.getAbsolutePath}, current version is ${AbortedTxn.CurrentVersion}") val nextEntry = (abortedTxn, position) position += AbortedTxn.TotalSize nextEntry @@ -147,7 +148,7 @@ class TransactionIndex(val startOffset: Long, @volatile var file: File) extends case e: IOException => // We received an unexpected error reading from the index file. We propagate this as an // UNKNOWN error to the consumer, which will cause it to retry the fetch. - throw new KafkaException(s"Failed to read from the transaction index $file", e) + throw new KafkaException(s"Failed to read from the transaction index ${file.getAbsolutePath}", e) } } } @@ -187,8 +188,8 @@ class TransactionIndex(val startOffset: Long, @volatile var file: File) extends val buffer = ByteBuffer.allocate(AbortedTxn.TotalSize) for ((abortedTxn, _) <- iterator(() => buffer)) { if (abortedTxn.lastOffset < startOffset) - throw new CorruptIndexException(s"Last offset of aborted transaction $abortedTxn is less than start offset " + - s"$startOffset") + throw new CorruptIndexException(s"Last offset of aborted transaction $abortedTxn in index " + + s"${file.getAbsolutePath} is less than start offset $startOffset") } } diff --git a/core/src/main/scala/kafka/message/ByteBufferMessageSet.scala b/core/src/main/scala/kafka/message/ByteBufferMessageSet.scala deleted file mode 100644 index 62e2125fabdb4..0000000000000 --- a/core/src/main/scala/kafka/message/ByteBufferMessageSet.scala +++ /dev/null @@ -1,197 +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.message - -import java.nio.ByteBuffer - -import kafka.common.LongRef -import kafka.utils.Logging -import org.apache.kafka.common.record._ - -import scala.collection.JavaConverters._ - -object ByteBufferMessageSet { - - private def create(offsetAssigner: OffsetAssigner, - compressionCodec: CompressionCodec, - timestampType: TimestampType, - messages: Message*): ByteBuffer = { - if (messages.isEmpty) - MessageSet.Empty.buffer - else { - val buffer = ByteBuffer.allocate(math.min(math.max(MessageSet.messageSetSize(messages) / 2, 1024), 1 << 16)) - val builder = MemoryRecords.builder(buffer, messages.head.magic, CompressionType.forId(compressionCodec.codec), - timestampType, offsetAssigner.baseOffset) - - for (message <- messages) - builder.appendWithOffset(offsetAssigner.nextAbsoluteOffset(), message.asRecord) - - builder.build().buffer - } - } - -} - -private object OffsetAssigner { - - def apply(offsetCounter: LongRef, size: Int): OffsetAssigner = - new OffsetAssigner(offsetCounter.value to offsetCounter.addAndGet(size)) - -} - -private class OffsetAssigner(offsets: Seq[Long]) { - private var index = 0 - - def nextAbsoluteOffset(): Long = { - val result = offsets(index) - index += 1 - result - } - - def baseOffset = offsets.head - - def toInnerOffset(offset: Long): Long = offset - offsets.head - -} - -/** - * A sequence of messages stored in a byte buffer - * - * There are two ways to create a 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. - * - * - * Message format v1 has the following changes: - * - For non-compressed messages, timestamp and timestamp type attributes have been added. The offsets of - * the messages remain absolute offsets. - * - For compressed messages, timestamp and timestamp type attributes have been added and inner offsets (IO) are used - * for inner messages of compressed messages (see offset calculation details below). The timestamp type - * attribute is only set in wrapper messages. Inner messages always have CreateTime as the timestamp type in attributes. - * - * We set the timestamp in the following way: - * For non-compressed messages: the timestamp and timestamp type message attributes are set and used. - * For compressed messages: - * 1. Wrapper messages' timestamp type attribute is set to the proper value - * 2. Wrapper messages' timestamp is set to: - * - the max timestamp of inner messages if CreateTime is used - * - the current server time if wrapper message's timestamp = LogAppendTime. - * In this case the wrapper message timestamp is used and all the timestamps of inner messages are ignored. - * 3. Inner messages' timestamp will be: - * - used when wrapper message's timestamp type is CreateTime - * - ignored when wrapper message's timestamp type is LogAppendTime - * 4. Inner messages' timestamp type will always be ignored with one exception: producers must set the inner message - * timestamp type to CreateTime, otherwise the messages will be rejected by broker. - * - * Absolute offsets are calculated in the following way: - * Ideally the conversion from relative offset(RO) to absolute offset(AO) should be: - * - * AO = AO_Of_Last_Inner_Message + RO - * - * However, note that the message sets sent by producers are compressed in a streaming way. - * And the relative offset of an inner message compared with the last inner message is not known until - * the last inner message is written. - * Unfortunately we are not able to change the previously written messages after the last message is written to - * the message set when stream compression is used. - * - * To solve this issue, we use the following solution: - * - * 1. When the producer creates a message set, it simply writes all the messages into a compressed message set with - * offset 0, 1, ... (inner offset). - * 2. The broker will set the offset of the wrapper message to the absolute offset of the last message in the - * message set. - * 3. When a consumer sees the message set, it first decompresses the entire message set to find out the inner - * offset (IO) of the last inner message. Then it computes RO and AO of previous messages: - * - * RO = IO_of_a_message - IO_of_the_last_message - * AO = AO_Of_Last_Inner_Message + RO - * - * 4. This solution works for compacted message sets as well. - * - */ -class ByteBufferMessageSet(val buffer: ByteBuffer) extends MessageSet with Logging { - - private[kafka] def this(compressionCodec: CompressionCodec, - offsetCounter: LongRef, - timestampType: TimestampType, - messages: Message*) { - this(ByteBufferMessageSet.create(OffsetAssigner(offsetCounter, messages.size), compressionCodec, - timestampType, messages:_*)) - } - - def this(compressionCodec: CompressionCodec, offsetCounter: LongRef, messages: Message*) { - this(compressionCodec, offsetCounter, TimestampType.CREATE_TIME, messages:_*) - } - - def this(compressionCodec: CompressionCodec, offsetSeq: Seq[Long], messages: Message*) { - this(ByteBufferMessageSet.create(new OffsetAssigner(offsetSeq), compressionCodec, - TimestampType.CREATE_TIME, messages:_*)) - } - - def this(compressionCodec: CompressionCodec, messages: Message*) { - this(compressionCodec, new LongRef(0L), messages: _*) - } - - def this(messages: Message*) { - this(NoCompressionCodec, messages: _*) - } - - def getBuffer = buffer - - override def asRecords: MemoryRecords = MemoryRecords.readableRecords(buffer.duplicate()) - - /** default iterator that iterates over decompressed messages */ - override def iterator: Iterator[MessageAndOffset] = internalIterator() - - /** iterator over compressed messages without decompressing */ - def shallowIterator: Iterator[MessageAndOffset] = internalIterator(isShallow = true) - - /** When flag isShallow is set to be true, we do a shallow iteration: just traverse the first level of messages. **/ - private def internalIterator(isShallow: Boolean = false): Iterator[MessageAndOffset] = { - if (isShallow) - asRecords.batches.asScala.iterator.map(MessageAndOffset.fromRecordBatch) - else - asRecords.records.asScala.iterator.map(MessageAndOffset.fromRecord) - } - - /** - * 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 - */ - def validBytes: Int = asRecords.validBytes - - /** - * Two message sets are equal if their respective byte buffers are equal - */ - override def equals(other: Any): Boolean = { - other match { - case that: ByteBufferMessageSet => - buffer.equals(that.buffer) - case _ => false - } - } - - override def hashCode: Int = buffer.hashCode - -} diff --git a/core/src/main/scala/kafka/message/CompressionCodec.scala b/core/src/main/scala/kafka/message/CompressionCodec.scala index a485271bcb72c..64e0aaa72a16a 100644 --- a/core/src/main/scala/kafka/message/CompressionCodec.scala +++ b/core/src/main/scala/kafka/message/CompressionCodec.scala @@ -19,6 +19,8 @@ package kafka.message import java.util.Locale +import kafka.common.UnknownCodecException + object CompressionCodec { def getCompressionCodec(codec: Int): CompressionCodec = { codec match { @@ -26,7 +28,7 @@ object CompressionCodec { case GZIPCompressionCodec.codec => GZIPCompressionCodec case SnappyCompressionCodec.codec => SnappyCompressionCodec case LZ4CompressionCodec.codec => LZ4CompressionCodec - case _ => throw new kafka.common.UnknownCodecException("%d is an unknown compression codec".format(codec)) + case _ => throw new UnknownCodecException("%d is an unknown compression codec".format(codec)) } } def getCompressionCodec(name: String): CompressionCodec = { diff --git a/core/src/main/scala/kafka/message/InvalidMessageException.scala b/core/src/main/scala/kafka/message/InvalidMessageException.scala deleted file mode 100644 index ef83500aea5ae..0000000000000 --- a/core/src/main/scala/kafka/message/InvalidMessageException.scala +++ /dev/null @@ -1,33 +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.message - -import org.apache.kafka.common.errors.CorruptRecordException - -/** - * Indicates that a message failed its checksum and is corrupt - * - * InvalidMessageException extends CorruptRecordException for temporary compatibility with the old Scala clients. - * We want to update the server side code to use and catch the new CorruptRecordException. - * Because ByteBufferMessageSet.scala and Message.scala are used in both server and client code having - * InvalidMessageException extend CorruptRecordException allows us to change server code without affecting the client. - */ -class InvalidMessageException(message: String, throwable: Throwable) extends CorruptRecordException(message, throwable) { - def this(message: String) = this(null, null) - def this() = this(null) -} diff --git a/core/src/main/scala/kafka/message/Message.scala b/core/src/main/scala/kafka/message/Message.scala deleted file mode 100755 index a46990160b6a1..0000000000000 --- a/core/src/main/scala/kafka/message/Message.scala +++ /dev/null @@ -1,379 +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.message - -import java.nio._ - -import org.apache.kafka.common.record.{CompressionType, LegacyRecord, TimestampType} - -import scala.math._ -import org.apache.kafka.common.utils.{ByteUtils, Crc32} - -/** - * Constants related to messages - */ -object Message { - - /** - * The current offset and size for all the fixed-length fields - */ - val CrcOffset = 0 - val CrcLength = 4 - val MagicOffset = CrcOffset + CrcLength - val MagicLength = 1 - val AttributesOffset = MagicOffset + MagicLength - val AttributesLength = 1 - // Only message format version 1 has the timestamp field. - val TimestampOffset = AttributesOffset + AttributesLength - val TimestampLength = 8 - val KeySizeOffset_V0 = AttributesOffset + AttributesLength - val KeySizeOffset_V1 = TimestampOffset + TimestampLength - val KeySizeLength = 4 - val KeyOffset_V0 = KeySizeOffset_V0 + KeySizeLength - val KeyOffset_V1 = KeySizeOffset_V1 + KeySizeLength - val ValueSizeLength = 4 - - private val MessageHeaderSizeMap = Map ( - (0: Byte) -> (CrcLength + MagicLength + AttributesLength + KeySizeLength + ValueSizeLength), - (1: Byte) -> (CrcLength + MagicLength + AttributesLength + TimestampLength + KeySizeLength + ValueSizeLength)) - - /** - * The amount of overhead bytes in a message - * This value is only used to check if the message size is valid or not. So the minimum possible message bytes is - * used here, which comes from a message in message format V0 with empty key and value. - */ - val MinMessageOverhead = KeyOffset_V0 + ValueSizeLength - - /** - * The "magic" value - * When magic value is 0, the message uses absolute offset and does not have a timestamp field. - * When magic value is 1, the message uses relative offset and has a timestamp field. - */ - val MagicValue_V0: Byte = 0 - val MagicValue_V1: Byte = 1 - val CurrentMagicValue: Byte = 1 - - /** - * Specifies the mask for the compression code. 3 bits to hold the compression codec. - * 0 is reserved to indicate no compression - */ - val CompressionCodeMask: Int = 0x07 - /** - * Specifies the mask for timestamp type. 1 bit at the 4th least significant bit. - * 0 for CreateTime, 1 for LogAppendTime - */ - val TimestampTypeMask: Byte = 0x08 - val TimestampTypeAttributeBitOffset: Int = 3 - - /** - * Compression code for uncompressed messages - */ - val NoCompression: Int = 0 - - /** - * To indicate timestamp is not defined so "magic" value 0 will be used. - */ - val NoTimestamp: Long = -1 - - /** - * Give the header size difference between different message versions. - */ - def headerSizeDiff(fromMagicValue: Byte, toMagicValue: Byte) : Int = - MessageHeaderSizeMap(toMagicValue) - MessageHeaderSizeMap(fromMagicValue) - - - def fromRecord(record: LegacyRecord): Message = { - val wrapperTimestamp: Option[Long] = if (record.wrapperRecordTimestamp == null) None else Some(record.wrapperRecordTimestamp) - val wrapperTimestampType = Option(record.wrapperRecordTimestampType) - new Message(record.buffer, wrapperTimestamp, wrapperTimestampType) - } -} - -/** - * 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 0 or 1 - * 3. 1 byte "attributes" identifier to allow annotations on the message independent of the version - * bit 0 ~ 2 : Compression codec. - * 0 : no compression - * 1 : gzip - * 2 : snappy - * 3 : lz4 - * bit 3 : Timestamp type - * 0 : create time - * 1 : log append time - * bit 4 ~ 7 : reserved - * 4. (Optional) 8 byte timestamp only if "magic" identifier is greater than 0 - * 5. 4 byte key length, containing length K - * 6. K byte key - * 7. 4 byte payload length, containing length V - * 8. V byte payload - * - * Default constructor wraps an existing ByteBuffer with the Message object with no change to the contents. - * @param buffer the byte buffer of this message. - * @param wrapperMessageTimestamp the wrapper message timestamp, which is only defined when the message is an inner - * message of a compressed message. - * @param wrapperMessageTimestampType the wrapper message timestamp type, which is only defined when the message is an - * inner message of a compressed message. - */ -class Message(val buffer: ByteBuffer, - private val wrapperMessageTimestamp: Option[Long] = None, - private val wrapperMessageTimestampType: Option[TimestampType] = None) { - - import kafka.message.Message._ - - private[message] def asRecord: LegacyRecord = wrapperMessageTimestamp match { - case None => new LegacyRecord(buffer) - case Some(timestamp) => new LegacyRecord(buffer, timestamp, wrapperMessageTimestampType.orNull) - } - - /** - * A constructor to create a Message - * @param bytes The payload of the message - * @param key The key of the message (null, if none) - * @param timestamp The timestamp of the message. - * @param timestampType The timestamp type of the message. - * @param codec The compression codec used on the contents of the message (if any) - * @param payloadOffset The offset into the payload array used to extract payload - * @param payloadSize The size of the payload to use - * @param magicValue the magic value to use - */ - def this(bytes: Array[Byte], - key: Array[Byte], - timestamp: Long, - timestampType: TimestampType, - codec: CompressionCodec, - payloadOffset: Int, - payloadSize: Int, - magicValue: Byte) = { - this(ByteBuffer.allocate(Message.CrcLength + - Message.MagicLength + - Message.AttributesLength + - (if (magicValue == Message.MagicValue_V0) 0 - else Message.TimestampLength) + - Message.KeySizeLength + - (if(key == null) 0 else key.length) + - Message.ValueSizeLength + - (if(bytes == null) 0 - else if(payloadSize >= 0) payloadSize - else bytes.length - payloadOffset))) - validateTimestampAndMagicValue(timestamp, magicValue) - // skip crc, we will fill that in at the end - buffer.position(MagicOffset) - buffer.put(magicValue) - val attributes: Byte = LegacyRecord.computeAttributes(magicValue, CompressionType.forId(codec.codec), timestampType) - buffer.put(attributes) - // Only put timestamp when "magic" value is greater than 0 - if (magic > MagicValue_V0) - buffer.putLong(timestamp) - if(key == null) { - buffer.putInt(-1) - } else { - buffer.putInt(key.length) - buffer.put(key, 0, key.length) - } - val size = if(bytes == null) -1 - else if(payloadSize >= 0) payloadSize - else bytes.length - payloadOffset - buffer.putInt(size) - if(bytes != null) - buffer.put(bytes, payloadOffset, size) - buffer.rewind() - - // now compute the checksum and fill it in - ByteUtils.writeUnsignedInt(buffer, CrcOffset, computeChecksum) - } - - def this(bytes: Array[Byte], key: Array[Byte], timestamp: Long, timestampType: TimestampType, codec: CompressionCodec, magicValue: Byte) = - this(bytes = bytes, key = key, timestamp = timestamp, timestampType = timestampType, codec = codec, payloadOffset = 0, payloadSize = -1, magicValue = magicValue) - - def this(bytes: Array[Byte], key: Array[Byte], timestamp: Long, codec: CompressionCodec, magicValue: Byte) = - this(bytes = bytes, key = key, timestamp = timestamp, timestampType = TimestampType.CREATE_TIME, codec = codec, payloadOffset = 0, payloadSize = -1, magicValue = magicValue) - - def this(bytes: Array[Byte], timestamp: Long, codec: CompressionCodec, magicValue: Byte) = - this(bytes = bytes, key = null, timestamp = timestamp, codec = codec, magicValue = magicValue) - - def this(bytes: Array[Byte], key: Array[Byte], timestamp: Long, magicValue: Byte) = - this(bytes = bytes, key = key, timestamp = timestamp, codec = NoCompressionCodec, magicValue = magicValue) - - def this(bytes: Array[Byte], timestamp: Long, magicValue: Byte) = - this(bytes = bytes, key = null, timestamp = timestamp, codec = NoCompressionCodec, magicValue = magicValue) - - def this(bytes: Array[Byte]) = - this(bytes = bytes, key = null, timestamp = Message.NoTimestamp, codec = NoCompressionCodec, magicValue = Message.CurrentMagicValue) - - /** - * Compute the checksum of the message from the message contents - */ - def computeChecksum: Long = - Crc32.crc32(buffer, MagicOffset, buffer.limit() - MagicOffset) - - /** - * Retrieve the previously computed CRC for this message - */ - def checksum: Long = ByteUtils.readUnsignedInt(buffer, CrcOffset) - - /** - * Returns true if the crc stored with the message matches the crc computed off the message contents - */ - def isValid: Boolean = checksum == computeChecksum - - /** - * Throw an InvalidMessageException if isValid is false for this message - */ - def ensureValid() { - if(!isValid) - throw new InvalidMessageException(s"Message is corrupt (stored crc = ${checksum}, computed crc = ${computeChecksum})") - } - - /** - * The complete serialized size of this message in bytes (including crc, header attributes, etc) - */ - def size: Int = buffer.limit() - - /** - * The position where the key size is stored. - */ - private def keySizeOffset = { - if (magic == MagicValue_V0) KeySizeOffset_V0 - else KeySizeOffset_V1 - } - - /** - * The length of the key in bytes - */ - def keySize: Int = buffer.getInt(keySizeOffset) - - /** - * Does the message have a key? - */ - def hasKey: Boolean = keySize >= 0 - - /** - * The position where the payload size is stored - */ - private def payloadSizeOffset = { - if (magic == MagicValue_V0) KeyOffset_V0 + max(0, keySize) - else KeyOffset_V1 + max(0, keySize) - } - - /** - * The length of the message value in bytes - */ - def payloadSize: Int = buffer.getInt(payloadSizeOffset) - - /** - * Is the payload of this message null - */ - def isNull: Boolean = payloadSize < 0 - - /** - * The magic version of this message - */ - def magic: Byte = buffer.get(MagicOffset) - - /** - * The attributes stored with this message - */ - def attributes: Byte = buffer.get(AttributesOffset) - - /** - * The timestamp of the message, only available when the "magic" value is greater than 0 - * When magic > 0, The timestamp of a message is determined in the following way: - * 1. wrapperMessageTimestampType = None and wrapperMessageTimestamp is None - Uncompressed message, timestamp and timestamp type are in the message. - * 2. wrapperMessageTimestampType = LogAppendTime and wrapperMessageTimestamp is defined - Compressed message using LogAppendTime - * 3. wrapperMessageTimestampType = CreateTime and wrapperMessageTimestamp is defined - Compressed message using CreateTime - */ - def timestamp: Long = { - if (magic == MagicValue_V0) - Message.NoTimestamp - // Case 2 - else if (wrapperMessageTimestampType.exists(_ == TimestampType.LOG_APPEND_TIME) && wrapperMessageTimestamp.isDefined) - wrapperMessageTimestamp.get - else // case 1, 3 - buffer.getLong(Message.TimestampOffset) - } - - /** - * The timestamp type of the message - */ - def timestampType = LegacyRecord.timestampType(magic, wrapperMessageTimestampType.orNull, attributes) - - /** - * The compression codec used with this message - */ - def compressionCodec: CompressionCodec = - CompressionCodec.getCompressionCodec(buffer.get(AttributesOffset) & CompressionCodeMask) - - /** - * A ByteBuffer containing the content of the message - */ - def payload: ByteBuffer = sliceDelimited(payloadSizeOffset) - - /** - * A ByteBuffer containing the message key - */ - def key: ByteBuffer = sliceDelimited(keySizeOffset) - - /** - * Read a size-delimited byte buffer starting at the given offset - */ - private def sliceDelimited(start: Int): ByteBuffer = { - val size = buffer.getInt(start) - if(size < 0) { - null - } else { - var b = buffer.duplicate() - b.position(start + 4) - b = b.slice() - b.limit(size) - b.rewind - b - } - } - - /** - * Validate the timestamp and "magic" value - */ - private def validateTimestampAndMagicValue(timestamp: Long, magic: Byte) { - if (magic != MagicValue_V0 && magic != MagicValue_V1) - throw new IllegalArgumentException(s"Invalid magic value $magic") - if (timestamp < 0 && timestamp != NoTimestamp) - throw new IllegalArgumentException(s"Invalid message timestamp $timestamp") - if (magic == MagicValue_V0 && timestamp != NoTimestamp) - throw new IllegalArgumentException(s"Invalid timestamp $timestamp. Timestamp must be $NoTimestamp when magic = $MagicValue_V0") - } - - override def toString: String = { - if (magic == MagicValue_V0) - s"Message(magic = $magic, attributes = $attributes, crc = $checksum, key = $key, payload = $payload)" - else - s"Message(magic = $magic, attributes = $attributes, $timestampType = $timestamp, crc = $checksum, key = $key, payload = $payload)" - } - - override def equals(any: Any): Boolean = { - any match { - case that: Message => this.buffer.equals(that.buffer) - case _ => false - } - } - - override def hashCode(): Int = buffer.hashCode - -} diff --git a/core/src/main/scala/kafka/message/MessageAndMetadata.scala b/core/src/main/scala/kafka/message/MessageAndMetadata.scala deleted file mode 100755 index 5c09cafdbac34..0000000000000 --- a/core/src/main/scala/kafka/message/MessageAndMetadata.scala +++ /dev/null @@ -1,39 +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.message - -import kafka.serializer.Decoder -import org.apache.kafka.common.record.TimestampType -import org.apache.kafka.common.utils.Utils - -case class MessageAndMetadata[K, V](topic: String, - partition: Int, - private val rawMessage: Message, - offset: Long, - keyDecoder: Decoder[K], valueDecoder: Decoder[V], - timestamp: Long = Message.NoTimestamp, - timestampType: TimestampType = TimestampType.CREATE_TIME) { - - /** - * Return the decoded message key and payload - */ - def key(): K = if(rawMessage.key == null) null.asInstanceOf[K] else keyDecoder.fromBytes(Utils.readBytes(rawMessage.key)) - - def message(): V = if(rawMessage.isNull) null.asInstanceOf[V] else valueDecoder.fromBytes(Utils.readBytes(rawMessage.payload)) -} - diff --git a/core/src/main/scala/kafka/message/MessageAndOffset.scala b/core/src/main/scala/kafka/message/MessageAndOffset.scala deleted file mode 100644 index 8de0f81bdfc61..0000000000000 --- a/core/src/main/scala/kafka/message/MessageAndOffset.scala +++ /dev/null @@ -1,54 +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.message - -import org.apache.kafka.common.record.{AbstractLegacyRecordBatch, Record, RecordBatch} - -object MessageAndOffset { - def fromRecordBatch(batch: RecordBatch): MessageAndOffset = { - batch match { - case legacyBatch: AbstractLegacyRecordBatch => - MessageAndOffset(Message.fromRecord(legacyBatch.outerRecord), legacyBatch.lastOffset) - - case _ => - throw new IllegalArgumentException(s"Illegal batch type ${batch.getClass}. The older message format classes " + - s"only support conversion from ${classOf[AbstractLegacyRecordBatch]}, which is used for magic v0 and v1") - } - } - - def fromRecord(record: Record): MessageAndOffset = { - record match { - case legacyBatch: AbstractLegacyRecordBatch => - MessageAndOffset(Message.fromRecord(legacyBatch.outerRecord), legacyBatch.lastOffset) - - case _ => - throw new IllegalArgumentException(s"Illegal record type ${record.getClass}. The older message format classes " + - s"only support conversion from ${classOf[AbstractLegacyRecordBatch]}, which is used for magic v0 and v1") - } - } -} - -case class MessageAndOffset(message: Message, offset: Long) { - - /** - * Compute the offset of the next message in the log - */ - def nextOffset: Long = offset + 1 - -} - diff --git a/core/src/main/scala/kafka/message/MessageLengthException.scala b/core/src/main/scala/kafka/message/MessageLengthException.scala deleted file mode 100644 index 45d32a5d4aed4..0000000000000 --- a/core/src/main/scala/kafka/message/MessageLengthException.scala +++ /dev/null @@ -1,24 +0,0 @@ -/** - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package kafka.message - -/** - * Indicates the presence of a message that exceeds the maximum acceptable - * length (whatever that happens to be) - */ -class MessageLengthException(message: String) extends RuntimeException(message) diff --git a/core/src/main/scala/kafka/message/MessageSet.scala b/core/src/main/scala/kafka/message/MessageSet.scala deleted file mode 100644 index 915def081a212..0000000000000 --- a/core/src/main/scala/kafka/message/MessageSet.scala +++ /dev/null @@ -1,111 +0,0 @@ -/** - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package kafka.message - -import java.nio._ - -import org.apache.kafka.common.record.Records - -/** - * Message set helper functions - */ -object MessageSet { - - val MessageSizeLength = 4 - val OffsetLength = 8 - val LogOverhead = MessageSizeLength + OffsetLength - val Empty = new ByteBufferMessageSet(ByteBuffer.allocate(0)) - - /** - * The size of a message set containing the given messages - */ - def messageSetSize(messages: Iterable[Message]): Int = - messages.foldLeft(0)(_ + entrySize(_)) - - /** - * The size of a size-delimited entry in a message set - */ - def entrySize(message: Message): Int = LogOverhead + message.size - - /** - * Validate that all "magic" values in `messages` are the same and return their magic value and max timestamp - */ - def magicAndLargestTimestamp(messages: Seq[Message]): MagicAndTimestamp = { - val firstMagicValue = messages.head.magic - var largestTimestamp = Message.NoTimestamp - for (message <- messages) { - if (message.magic != firstMagicValue) - throw new IllegalStateException("Messages in the same message set must have same magic value") - if (firstMagicValue > Message.MagicValue_V0) - largestTimestamp = math.max(largestTimestamp, message.timestamp) - } - MagicAndTimestamp(firstMagicValue, largestTimestamp) - } - -} - -case class MagicAndTimestamp(magic: Byte, timestamp: Long) - -/** - * A set of messages with offsets. A message set has a fixed serialized form, though the container - * for the bytes could be either in-memory or on disk. The format of each message is - * as follows: - * 8 byte message offset number - * 4 byte size containing an integer N - * N message bytes as described in the Message class - */ -abstract class MessageSet extends Iterable[MessageAndOffset] { - - /** - * Provides an iterator over the message/offset pairs in this set - */ - def iterator: Iterator[MessageAndOffset] - - /** - * Gives the total size of this message set in bytes - */ - def sizeInBytes: Int - - /** - * Get the client representation of the message set - */ - def asRecords: Records - - /** - * Print this message set's contents. If the message set has more than 100 messages, just - * print the first 100. - */ - override def toString: String = { - val builder = new StringBuilder() - builder.append(getClass.getSimpleName + "(") - val iter = this.asRecords.batches.iterator - var i = 0 - while(iter.hasNext && i < 100) { - val message = iter.next - builder.append(message) - if(iter.hasNext) - builder.append(", ") - i += 1 - } - if(iter.hasNext) - builder.append("...") - builder.append(")") - builder.toString - } - -} diff --git a/core/src/main/scala/kafka/message/package.html b/core/src/main/scala/kafka/message/package.html deleted file mode 100644 index c666d81d374bc..0000000000000 --- a/core/src/main/scala/kafka/message/package.html +++ /dev/null @@ -1,19 +0,0 @@ - -Messages and everything related to them. \ No newline at end of file diff --git a/core/src/main/scala/kafka/metrics/KafkaMetricsGroup.scala b/core/src/main/scala/kafka/metrics/KafkaMetricsGroup.scala index f95d0ad5b16e5..03a4f7ccd4c03 100644 --- a/core/src/main/scala/kafka/metrics/KafkaMetricsGroup.scala +++ b/core/src/main/scala/kafka/metrics/KafkaMetricsGroup.scala @@ -21,14 +21,9 @@ import java.util.concurrent.TimeUnit import com.yammer.metrics.Metrics import com.yammer.metrics.core.{Gauge, MetricName} -import kafka.consumer.{ConsumerTopicStatsRegistry, FetchRequestAndResponseStatsRegistry} import kafka.utils.Logging import org.apache.kafka.common.utils.Sanitizer -import scala.collection.immutable -import scala.collection.JavaConverters._ - - trait KafkaMetricsGroup extends Logging { /** @@ -63,8 +58,8 @@ trait KafkaMetricsGroup extends Logging { nameBuilder.append(name) } - val scope: String = KafkaMetricsGroup.toScope(tags).getOrElse(null) - val tagsName = KafkaMetricsGroup.toMBeanName(tags) + val scope: String = toScope(tags).getOrElse(null) + val tagsName = toMBeanName(tags) tagsName.foreach(nameBuilder.append(",").append(_)) new MetricName(group, typeName, name, scope, nameBuilder.toString) @@ -85,72 +80,6 @@ trait KafkaMetricsGroup extends Logging { def removeMetric(name: String, tags: scala.collection.Map[String, String] = Map.empty) = Metrics.defaultRegistry().removeMetric(metricName(name, tags)) - -} - -object KafkaMetricsGroup extends KafkaMetricsGroup with Logging { - /** - * To make sure all the metrics be de-registered after consumer/producer close, the metric names should be - * put into the metric name set. - */ - private val consumerMetricNameList: immutable.List[MetricName] = immutable.List[MetricName]( - // kafka.consumer.ZookeeperConsumerConnector - new MetricName("kafka.consumer", "ZookeeperConsumerConnector", "FetchQueueSize"), - new MetricName("kafka.consumer", "ZookeeperConsumerConnector", "KafkaCommitsPerSec"), - new MetricName("kafka.consumer", "ZookeeperConsumerConnector", "ZooKeeperCommitsPerSec"), - new MetricName("kafka.consumer", "ZookeeperConsumerConnector", "RebalanceRateAndTime"), - new MetricName("kafka.consumer", "ZookeeperConsumerConnector", "OwnedPartitionsCount"), - - // kafka.consumer.ConsumerFetcherManager - new MetricName("kafka.consumer", "ConsumerFetcherManager", "MaxLag"), - new MetricName("kafka.consumer", "ConsumerFetcherManager", "MinFetchRate"), - - // kafka.server.AbstractFetcherThread <-- kafka.consumer.ConsumerFetcherThread - new MetricName("kafka.server", "FetcherLagMetrics", "ConsumerLag"), - - // kafka.consumer.ConsumerTopicStats <-- kafka.consumer.{ConsumerIterator, PartitionTopicInfo} - new MetricName("kafka.consumer", "ConsumerTopicMetrics", "MessagesPerSec"), - - // kafka.consumer.ConsumerTopicStats - new MetricName("kafka.consumer", "ConsumerTopicMetrics", "BytesPerSec"), - - // kafka.server.AbstractFetcherThread <-- kafka.consumer.ConsumerFetcherThread - new MetricName("kafka.server", "FetcherStats", "BytesPerSec"), - new MetricName("kafka.server", "FetcherStats", "RequestsPerSec"), - - // kafka.consumer.FetchRequestAndResponseStats <-- kafka.consumer.SimpleConsumer - new MetricName("kafka.consumer", "FetchRequestAndResponseMetrics", "FetchResponseSize"), - new MetricName("kafka.consumer", "FetchRequestAndResponseMetrics", "FetchRequestRateAndTimeMs"), - 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") - ) - - private val producerMetricNameList: immutable.List[MetricName] = immutable.List[MetricName]( - // kafka.producer.ProducerStats <-- DefaultEventHandler <-- Producer - new MetricName("kafka.producer", "ProducerStats", "SerializationErrorsPerSec"), - new MetricName("kafka.producer", "ProducerStats", "ResendsPerSec"), - new MetricName("kafka.producer", "ProducerStats", "FailedSendsPerSec"), - - // kafka.producer.ProducerSendThread - new MetricName("kafka.producer.async", "ProducerSendThread", "ProducerQueueSize"), - - // kafka.producer.ProducerTopicStats <-- kafka.producer.{Producer, async.DefaultEventHandler} - new MetricName("kafka.producer", "ProducerTopicMetrics", "MessagesPerSec"), - new MetricName("kafka.producer", "ProducerTopicMetrics", "DroppedMessagesPerSec"), - new MetricName("kafka.producer", "ProducerTopicMetrics", "BytesPerSec"), - - // kafka.producer.ProducerRequestStats <-- SyncProducer - new MetricName("kafka.producer", "ProducerRequestMetrics", "ProducerRequestRateAndTimeMs"), - new MetricName("kafka.producer", "ProducerRequestMetrics", "ProducerRequestSize"), - new MetricName("kafka.producer", "ProducerRequestMetrics", "ProducerRequestThrottleRateAndTimeMs") - ) - private def toMBeanName(tags: collection.Map[String, String]): Option[String] = { val filteredTags = tags.filter { case (_, tagValue) => tagValue != "" } if (filteredTags.nonEmpty) { @@ -174,33 +103,6 @@ object KafkaMetricsGroup extends KafkaMetricsGroup with Logging { else None } - @deprecated("This method has been deprecated and will be removed in a future release.", "0.11.0.0") - def removeAllConsumerMetrics(clientId: String) { - FetchRequestAndResponseStatsRegistry.removeConsumerFetchRequestAndResponseStats(clientId) - ConsumerTopicStatsRegistry.removeConsumerTopicStat(clientId) - removeAllMetricsInList(KafkaMetricsGroup.consumerMetricNameList, clientId) - } - - private def removeAllMetricsInList(metricNameList: immutable.List[MetricName], clientId: String) { - metricNameList.foreach(metric => { - val pattern = (".*clientId=" + clientId + ".*").r - val registeredMetrics = Metrics.defaultRegistry().allMetrics().keySet().asScala - for (registeredMetric <- registeredMetrics) { - if (registeredMetric.getGroup == metric.getGroup && - registeredMetric.getName == metric.getName && - registeredMetric.getType == metric.getType) { - pattern.findFirstIn(registeredMetric.getMBeanName) match { - case Some(_) => { - val beforeRemovalSize = Metrics.defaultRegistry().allMetrics().keySet().size - Metrics.defaultRegistry().removeMetric(registeredMetric) - val afterRemovalSize = Metrics.defaultRegistry().allMetrics().keySet().size - trace("Removing metric %s. Metrics registry size reduced from %d to %d".format( - registeredMetric, beforeRemovalSize, afterRemovalSize)) - } - case _ => - } - } - } - }) - } } + +object KafkaMetricsGroup extends KafkaMetricsGroup diff --git a/core/src/main/scala/kafka/network/BlockingChannel.scala b/core/src/main/scala/kafka/network/BlockingChannel.scala deleted file mode 100644 index 3493ad34e1852..0000000000000 --- a/core/src/main/scala/kafka/network/BlockingChannel.scala +++ /dev/null @@ -1,135 +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.net.InetSocketAddress -import java.nio.channels._ - -import kafka.api.RequestOrResponse -import kafka.utils.{CoreUtils, Logging, nonthreadsafe} -import org.apache.kafka.common.network.NetworkReceive - - -@deprecated("This object has been deprecated and will be removed in a future release.", "0.11.0.0") -object BlockingChannel{ - val UseDefaultBufferSize = -1 -} - -/** - * A simple blocking channel with timeouts correctly enabled. - * - */ -@nonthreadsafe -@deprecated("This class has been deprecated and will be removed in a future release.", "0.11.0.0") -class BlockingChannel( val host: String, - val port: Int, - val readBufferSize: Int, - val writeBufferSize: Int, - val readTimeoutMs: Int ) extends Logging { - private var connected = false - private var channel: SocketChannel = null - 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 { - channel = SocketChannel.open() - if(readBufferSize > 0) - channel.socket.setReceiveBufferSize(readBufferSize) - if(writeBufferSize > 0) - channel.socket.setSendBufferSize(writeBufferSize) - channel.configureBlocking(true) - channel.socket.setSoTimeout(readTimeoutMs) - channel.socket.setKeepAlive(true) - channel.socket.setTcpNoDelay(true) - channel.socket.connect(new InetSocketAddress(host, port), connectTimeoutMs) - - writeChannel = channel - // Need to create a new ReadableByteChannel from input stream because SocketChannel doesn't implement read with timeout - // See: http://stackoverflow.com/questions/2866557/timeout-for-socketchannel-doesnt-work - readChannel = Channels.newChannel(channel.socket().getInputStream) - connected = true - val localHost = channel.socket.getLocalAddress.getHostAddress - val localPort = channel.socket.getLocalPort - val remoteHost = channel.socket.getInetAddress.getHostAddress - val remotePort = channel.socket.getPort - connectionId = localHost + ":" + localPort + "-" + remoteHost + ":" + remotePort - // settings may not match what we requested above - val msg = "Created socket with SO_TIMEOUT = %d (requested %d), SO_RCVBUF = %d (requested %d), SO_SNDBUF = %d (requested %d), connectTimeoutMs = %d." - debug(msg.format(channel.socket.getSoTimeout, - readTimeoutMs, - channel.socket.getReceiveBufferSize, - readBufferSize, - channel.socket.getSendBufferSize, - writeBufferSize, - connectTimeoutMs)) - - } catch { - case _: Throwable => disconnect() - } - } - } - - def disconnect() = lock synchronized { - if(channel != null) { - CoreUtils.swallow(channel.close(), this) - CoreUtils.swallow(channel.socket.close(), this) - channel = null - writeChannel = null - } - // closing the main socket channel *should* close the read channel - // but let's do it to be sure. - if(readChannel != null) { - CoreUtils.swallow(readChannel.close(), this) - readChannel = null - } - connected = false - } - - def isConnected = connected - - def send(request: RequestOrResponse): Long = { - if(!connected) - throw new ClosedChannelException() - - val send = new RequestOrResponseSend(connectionId, request) - send.writeCompletely(writeChannel) - } - - def receive(): NetworkReceive = { - if(!connected) - throw new ClosedChannelException() - - 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/RequestChannel.scala b/core/src/main/scala/kafka/network/RequestChannel.scala index eecce1d14151d..00b09688c5b2c 100644 --- a/core/src/main/scala/kafka/network/RequestChannel.scala +++ b/core/src/main/scala/kafka/network/RequestChannel.scala @@ -24,12 +24,10 @@ import java.util.concurrent._ import com.typesafe.scalalogging.Logger import com.yammer.metrics.core.{Gauge, Meter} import kafka.metrics.KafkaMetricsGroup -import kafka.utils.{Logging, NotNothing} -import org.apache.kafka.common.TopicPartition +import kafka.utils.{Logging, NotNothing, Pool} import org.apache.kafka.common.memory.MemoryPool import org.apache.kafka.common.network.Send import org.apache.kafka.common.protocol.{ApiKeys, Errors} -import org.apache.kafka.common.record.RecordConversionStats import org.apache.kafka.common.requests._ import org.apache.kafka.common.security.auth.KafkaPrincipal import org.apache.kafka.common.utils.{Sanitizer, Time} @@ -389,7 +387,7 @@ class RequestMetrics(name: String) extends KafkaMetricsGroup { import RequestMetrics._ val tags = Map("request" -> name) - val requestRateInternal = new mutable.HashMap[Short, Meter] + val requestRateInternal = new Pool[Short, Meter]() // time a request spent in a request queue val requestQueueTimeHist = newHistogram(RequestQueueTimeMs, biased = true, tags) // time a request takes to be processed at the local broker @@ -423,7 +421,7 @@ class RequestMetrics(name: String) extends KafkaMetricsGroup { Errors.values.foreach(error => errorMeters.put(error, new ErrorMeter(name, error))) def requestRate(version: Short): Meter = { - requestRateInternal.getOrElseUpdate(version, newMeter("RequestsPerSec", "requests", TimeUnit.SECONDS, tags + ("version" -> version.toString))) + requestRateInternal.getAndMaybePut(version, newMeter("RequestsPerSec", "requests", TimeUnit.SECONDS, tags + ("version" -> version.toString))) } class ErrorMeter(name: String, error: Errors) { @@ -458,7 +456,7 @@ class RequestMetrics(name: String) extends KafkaMetricsGroup { } def removeMetrics(): Unit = { - for (version <- requestRateInternal.keySet) removeMetric(RequestsPerSec, tags + ("version" -> version.toString)) + for (version <- requestRateInternal.keys) removeMetric(RequestsPerSec, tags + ("version" -> version.toString)) removeMetric(RequestQueueTimeMs, tags) removeMetric(LocalTimeMs, tags) removeMetric(RemoteTimeMs, tags) diff --git a/core/src/main/scala/kafka/network/RequestOrResponseSend.scala b/core/src/main/scala/kafka/network/RequestOrResponseSend.scala deleted file mode 100644 index 7a14e5e7cacec..0000000000000 --- a/core/src/main/scala/kafka/network/RequestOrResponseSend.scala +++ /dev/null @@ -1,53 +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.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 + request.requestId.fold(0)(_ => 2)) - request.requestId.foreach(buffer.putShort) - 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 db5eda60b4f85..81b1474d5f763 100644 --- a/core/src/main/scala/kafka/network/SocketServer.scala +++ b/core/src/main/scala/kafka/network/SocketServer.scala @@ -26,19 +26,17 @@ import java.util.concurrent.atomic._ import com.yammer.metrics.core.Gauge import kafka.cluster.{BrokerEndPoint, EndPoint} -import kafka.common.KafkaException import kafka.metrics.KafkaMetricsGroup import kafka.network.RequestChannel.{CloseConnectionResponse, EndThrottlingResponse, NoOpResponse, SendResponse, StartThrottlingResponse} import kafka.security.CredentialProvider import kafka.server.KafkaConfig import kafka.utils._ -import org.apache.kafka.common.Reconfigurable +import org.apache.kafka.common.{KafkaException, Reconfigurable} import org.apache.kafka.common.memory.{MemoryPool, SimpleMemoryPool} import org.apache.kafka.common.metrics._ import org.apache.kafka.common.metrics.stats.Meter import org.apache.kafka.common.network.KafkaChannel.ChannelMuteEvent import org.apache.kafka.common.network.{ChannelBuilder, ChannelBuilders, KafkaChannel, ListenerName, Selectable, Send, Selector => KSelector} -import org.apache.kafka.common.record.MultiRecordsSend import org.apache.kafka.common.requests.{RequestContext, RequestHeader} import org.apache.kafka.common.security.auth.SecurityProtocol import org.apache.kafka.common.utils.{KafkaThread, LogContext, Time} @@ -107,7 +105,7 @@ class SocketServer(val config: KafkaConfig, val metrics: Metrics, val time: Time metrics.metricName("io-wait-ratio", "socket-server-metrics", p.metricTags) } ioWaitRatioMetricNames.map { metricName => - Option(metrics.metric(metricName)).fold(0.0)(_.value) + Option(metrics.metric(metricName)).fold(0.0)(m => Math.min(m.metricValue.asInstanceOf[Double], 1.0)) }.sum / processors.size } } @@ -533,7 +531,8 @@ private[kafka] class Processor(val id: Int, newGauge(IdlePercentMetricName, new Gauge[Double] { def value = { - Option(metrics.metric(metrics.metricName("io-wait-ratio", "socket-server-metrics", metricTags))).fold(0.0)(_.value) + Option(metrics.metric(metrics.metricName("io-wait-ratio", "socket-server-metrics", metricTags))) + .fold(0.0)(m => Math.min(m.metricValue.asInstanceOf[Double], 1.0)) } }, // for compatibility, only add a networkProcessor tag to the Yammer Metrics alias (the equivalent Selector metric @@ -641,9 +640,9 @@ private[kafka] class Processor(val id: Int, updateRequestMetrics(response) trace("Closing socket connection actively according to the response code.") close(channelId) - case response: StartThrottlingResponse => + case _: StartThrottlingResponse => handleChannelMuteEvent(channelId, ChannelMuteEvent.THROTTLE_STARTED) - case response: EndThrottlingResponse => + case _: EndThrottlingResponse => // Try unmuting the channel. The channel will be unmuted only if the response has already been sent out to // the client. handleChannelMuteEvent(channelId, ChannelMuteEvent.THROTTLE_ENDED) diff --git a/core/src/main/scala/kafka/security/SecurityUtils.scala b/core/src/main/scala/kafka/security/SecurityUtils.scala index 8442ba0a36fe8..74bd4043edb9f 100644 --- a/core/src/main/scala/kafka/security/SecurityUtils.scala +++ b/core/src/main/scala/kafka/security/SecurityUtils.scala @@ -17,11 +17,11 @@ package kafka.security -import kafka.security.auth.{Acl, Operation, PermissionType, Resource, ResourceNameType, ResourceType} +import kafka.security.auth.{Acl, Operation, PermissionType, Resource, ResourceType} import org.apache.kafka.common.acl.{AccessControlEntry, AclBinding, AclBindingFilter} import org.apache.kafka.common.protocol.Errors import org.apache.kafka.common.requests.ApiError -import org.apache.kafka.common.resource.{Resource => AdminResource} +import org.apache.kafka.common.resource.ResourcePattern import org.apache.kafka.common.security.auth.KafkaPrincipal import scala.util.{Failure, Success, Try} @@ -31,12 +31,11 @@ object SecurityUtils { def convertToResourceAndAcl(filter: AclBindingFilter): Either[ApiError, (Resource, Acl)] = { (for { - resourceType <- Try(ResourceType.fromJava(filter.resourceFilter.resourceType)) - resourceNameType <- Try(ResourceNameType.fromJava(filter.resourceFilter.nameType)) + resourceType <- Try(ResourceType.fromJava(filter.patternFilter.resourceType)) principal <- Try(KafkaPrincipal.fromString(filter.entryFilter.principal)) operation <- Try(Operation.fromJava(filter.entryFilter.operation)) permissionType <- Try(PermissionType.fromJava(filter.entryFilter.permissionType)) - resource = Resource(resourceType, filter.resourceFilter.name, resourceNameType) + resource = Resource(resourceType, filter.patternFilter.name, filter.patternFilter.patternType) acl = Acl(principal, permissionType, filter.entryFilter.host, operation) } yield (resource, acl)) match { case Failure(throwable) => Left(new ApiError(Errors.INVALID_REQUEST, throwable.getMessage)) @@ -45,10 +44,10 @@ object SecurityUtils { } def convertToAclBinding(resource: Resource, acl: Acl): AclBinding = { - val adminResource = new AdminResource(resource.resourceType.toJava, resource.name, resource.resourceNameType.toJava) + val resourcePattern = new ResourcePattern(resource.resourceType.toJava, resource.name, resource.patternType) val entry = new AccessControlEntry(acl.principal.toString, acl.host.toString, acl.operation.toJava, acl.permissionType.toJava) - new AclBinding(adminResource, entry) + new AclBinding(resourcePattern, entry) } } diff --git a/core/src/main/scala/kafka/security/auth/Acl.scala b/core/src/main/scala/kafka/security/auth/Acl.scala index 7fa1638cf0d59..bdd8584036061 100644 --- a/core/src/main/scala/kafka/security/auth/Acl.scala +++ b/core/src/main/scala/kafka/security/auth/Acl.scala @@ -18,6 +18,7 @@ package kafka.security.auth import kafka.utils.Json +import org.apache.kafka.common.resource.ResourcePattern import org.apache.kafka.common.security.auth.KafkaPrincipal import org.apache.kafka.common.utils.SecurityUtils @@ -26,7 +27,7 @@ import scala.collection.JavaConverters._ object Acl { val WildCardPrincipal: KafkaPrincipal = new KafkaPrincipal(KafkaPrincipal.USER_TYPE, "*") val WildCardHost: String = "*" - val WildCardResource: String = org.apache.kafka.common.resource.Resource.WILDCARD_RESOURCE + val WildCardResource: String = ResourcePattern.WILDCARD_RESOURCE val AllowAllAcl = new Acl(WildCardPrincipal, Allow, WildCardHost, All) val PrincipalKey = "principal" val PermissionTypeKey = "permissionType" diff --git a/core/src/main/scala/kafka/security/auth/Authorizer.scala b/core/src/main/scala/kafka/security/auth/Authorizer.scala index 4f4ddcfa2d09c..9be8e6c704930 100644 --- a/core/src/main/scala/kafka/security/auth/Authorizer.scala +++ b/core/src/main/scala/kafka/security/auth/Authorizer.scala @@ -37,7 +37,7 @@ 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. Resource name type is always literal in input resource. + * @param resource Resource the client is trying to access. Resource pattern type is always literal in input resource. * @return true if the operation should be permitted, false otherwise */ def authorize(session: Session, operation: Operation, resource: Resource): Boolean @@ -48,17 +48,19 @@ trait Authorizer extends Configurable { * * {code} * // The following will add ACLs to the literal resource path 'foo', which will only affect the topic named 'foo': - * authorizer.addAcls(Set(acl1, acl2), Resource(Topic, "foo", Literal)) + * authorizer.addAcls(Set(acl1, acl2), Resource(Topic, "foo", LITERAL)) * * // The following will add ACLs to the special literal topic resource path '*', which affects all topics: - * authorizer.addAcls(Set(acl1, acl2), Resource(Topic, "*", Literal)) + * authorizer.addAcls(Set(acl1, acl2), Resource(Topic, "*", LITERAL)) * * // The following will add ACLs to the prefixed resource path 'foo', which affects all topics whose name begins with 'foo': - * authorizer.addAcls(Set(acl1, acl2), Resource(Topic, "foo", Prefixed)) + * authorizer.addAcls(Set(acl1, acl2), Resource(Topic, "foo", PREFIXED)) * {code} * * @param acls set of acls to add to existing acls - * @param resource the resource path to which these acls should be attached + * @param resource the resource path to which these acls should be attached. + * the supplied resource will have a specific resource pattern type, + * i.e. the resource pattern type will not be ``PatternType.ANY`` or ``PatternType.UNKNOWN``. */ def addAcls(acls: Set[Acl], resource: Resource): Unit @@ -67,17 +69,19 @@ trait Authorizer extends Configurable { * * {code} * // The following will remove ACLs from the literal resource path 'foo', which will only affect the topic named 'foo': - * authorizer.removeAcls(Set(acl1, acl2), Resource(Topic, "foo", Literal)) + * authorizer.removeAcls(Set(acl1, acl2), Resource(Topic, "foo", LITERAL)) * * // The following will remove ACLs from the special literal topic resource path '*', which affects all topics: - * authorizer.removeAcls(Set(acl1, acl2), Resource(Topic, "*", Literal)) + * authorizer.removeAcls(Set(acl1, acl2), Resource(Topic, "*", LITERAL)) * * // The following will remove ACLs from the prefixed resource path 'foo', which affects all topics whose name begins with 'foo': - * authorizer.removeAcls(Set(acl1, acl2), Resource(Topic, "foo", Prefixed)) + * authorizer.removeAcls(Set(acl1, acl2), Resource(Topic, "foo", PREFIXED)) * {code} * * @param acls set of acls to be removed. * @param resource resource path from which the acls should be removed. + * the supplied resource will have a specific resource pattern type, + * i.e. the resource pattern type will not be ``PatternType.ANY`` or ``PatternType.UNKNOWN``. * @return true if some acl got removed, false if no acl was removed. */ def removeAcls(acls: Set[Acl], resource: Resource): Boolean @@ -87,16 +91,18 @@ trait Authorizer extends Configurable { * * {code} * // The following will remove all ACLs from the literal resource path 'foo', which will only affect the topic named 'foo': - * authorizer.removeAcls(Resource(Topic, "foo", Literal)) + * authorizer.removeAcls(Resource(Topic, "foo", LITERAL)) * * // The following will remove all ACLs from the special literal topic resource path '*', which affects all topics: - * authorizer.removeAcls(Resource(Topic, "*", Literal)) + * authorizer.removeAcls(Resource(Topic, "*", LITERAL)) * * // The following will remove all ACLs from the prefixed resource path 'foo', which affects all topics whose name begins with 'foo': - * authorizer.removeAcls(Resource(Topic, "foo", Prefixed)) + * authorizer.removeAcls(Resource(Topic, "foo", PREFIXED)) * {code} * * @param resource the resource path from which these acls should be removed. + * the supplied resource will have a specific resource pattern type, + * i.e. the resource pattern type will not be ``PatternType.ANY`` or ``PatternType.UNKNOWN``. * @return */ def removeAcls(resource: Resource): Boolean @@ -106,16 +112,18 @@ trait Authorizer extends Configurable { * * {code} * // The following will get all ACLs from the literal resource path 'foo', which will only affect the topic named 'foo': - * authorizer.removeAcls(Resource(Topic, "foo", Literal)) + * authorizer.removeAcls(Resource(Topic, "foo", LITERAL)) * * // The following will get all ACLs from the special literal topic resource path '*', which affects all topics: - * authorizer.removeAcls(Resource(Topic, "*", Literal)) + * authorizer.removeAcls(Resource(Topic, "*", LITERAL)) * * // The following will get all ACLs from the prefixed resource path 'foo', which affects all topics whose name begins with 'foo': - * authorizer.removeAcls(Resource(Topic, "foo", Prefixed)) + * authorizer.removeAcls(Resource(Topic, "foo", PREFIXED)) * {code} * * @param resource the resource path to which the acls belong. + * the supplied resource will have a specific resource pattern type, + * i.e. the resource pattern type will not be ``PatternType.ANY`` or ``PatternType.UNKNOWN``. * @return empty set if no acls are found, otherwise the acls for the resource. */ def getAcls(resource: Resource): Set[Acl] diff --git a/core/src/main/scala/kafka/security/auth/Resource.scala b/core/src/main/scala/kafka/security/auth/Resource.scala index fa63fccf10f8a..c4755961f94a5 100644 --- a/core/src/main/scala/kafka/security/auth/Resource.scala +++ b/core/src/main/scala/kafka/security/auth/Resource.scala @@ -16,14 +16,32 @@ */ package kafka.security.auth -import java.util.Objects -import org.apache.kafka.common.resource.{Resource => JResource} +import kafka.common.KafkaException +import org.apache.kafka.common.resource.{PatternType, ResourcePattern} object Resource { + val Separator = ":" val ClusterResourceName = "kafka-cluster" - val ClusterResource = new Resource(Cluster, Resource.ClusterResourceName, Literal) + val ClusterResource = Resource(Cluster, Resource.ClusterResourceName, PatternType.LITERAL) val ProducerIdResourceName = "producer-id" val WildCardResource = "*" + + def fromString(str: String): Resource = { + ResourceType.values.find(resourceType => str.startsWith(resourceType.name + Separator)) match { + case None => throw new KafkaException("Invalid resource string: '" + str + "'") + case Some(resourceType) => + val remaining = str.substring(resourceType.name.length + 1) + + PatternType.values.find(patternType => remaining.startsWith(patternType.name + Separator)) match { + case Some(patternType) => + val name = remaining.substring(patternType.name.length + 1) + Resource(resourceType, name, patternType) + + case None => + Resource(resourceType, remaining, PatternType.LITERAL) + } + } + } } /** @@ -31,29 +49,35 @@ object Resource { * @param resourceType non-null type of resource. * @param name non-null 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. - * @param resourceNameType non-null type of resource name: literal, prefixed, etc. + * @param patternType non-null resource pattern type: literal, prefixed, etc. */ -case class Resource(resourceType: ResourceType, name: String, resourceNameType: ResourceNameType) { +case class Resource(resourceType: ResourceType, name: String, patternType: PatternType) { - Objects.requireNonNull(resourceType, "resourceType") - Objects.requireNonNull(name, "name") - Objects.requireNonNull(resourceNameType, "resourceNameType") + if (patternType == PatternType.MATCH || patternType == PatternType.ANY) + throw new IllegalArgumentException("patternType must not be " + patternType) + + if (patternType == PatternType.UNKNOWN) + throw new IllegalArgumentException("patternType must not be UNKNOWN") /** * Create an instance of this class with the provided parameters. - * Resource name type would default to ResourceNameType.LITERAL. + * Resource pattern type would default to PatternType.LITERAL. * * @param resourceType non-null resource type * @param name non-null resource name - * @deprecated Since 2.0, use [[kafka.security.auth.Resource(ResourceType, String, ResourceNameType)]] + * @deprecated Since 2.0, use [[kafka.security.auth.Resource(ResourceType, String, PatternType)]] */ - @deprecated("Use Resource(ResourceType, String, ResourceNameType") + @deprecated("Use Resource(ResourceType, String, PatternType", "Since 2.0") def this(resourceType: ResourceType, name: String) { - this(resourceType, name, Literal) + this(resourceType, name, PatternType.LITERAL) + } + + def toPattern: ResourcePattern = { + new ResourcePattern(resourceType.toJava, name, patternType) } - def toJava: JResource = { - new JResource(resourceType.toJava, name, resourceNameType.toJava) + override def toString: String = { + resourceType.name + Resource.Separator + patternType + Resource.Separator + name } } diff --git a/core/src/main/scala/kafka/security/auth/ResourceNameType.scala b/core/src/main/scala/kafka/security/auth/ResourceNameType.scala deleted file mode 100644 index 21b10a1ef0668..0000000000000 --- a/core/src/main/scala/kafka/security/auth/ResourceNameType.scala +++ /dev/null @@ -1,49 +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.security.auth - -import kafka.common.{BaseEnum, KafkaException} -import org.apache.kafka.common.resource.{ResourceNameType => JResourceNameType} - -sealed trait ResourceNameType extends BaseEnum with Ordered[ ResourceNameType ] { - def toJava: JResourceNameType - - override def compare(that: ResourceNameType): Int = this.name compare that.name -} - -case object Literal extends ResourceNameType { - val name = "Literal" - val toJava = JResourceNameType.LITERAL -} - -case object Prefixed extends ResourceNameType { - val name = "Prefixed" - val toJava = JResourceNameType.PREFIXED -} - -object ResourceNameType { - - def fromString(resourceNameType: String): ResourceNameType = { - val rType = values.find(rType => rType.name.equalsIgnoreCase(resourceNameType)) - rType.getOrElse(throw new KafkaException(resourceNameType + " not a valid resourceNameType name. The valid names are " + values.mkString(","))) - } - - def values: Seq[ResourceNameType] = List(Literal, Prefixed) - - def fromJava(nameType: JResourceNameType): ResourceNameType = fromString(nameType.toString) -} diff --git a/core/src/main/scala/kafka/security/auth/SimpleAclAuthorizer.scala b/core/src/main/scala/kafka/security/auth/SimpleAclAuthorizer.scala index c828970cc0d79..b64c8c2b9a1e6 100644 --- a/core/src/main/scala/kafka/security/auth/SimpleAclAuthorizer.scala +++ b/core/src/main/scala/kafka/security/auth/SimpleAclAuthorizer.scala @@ -20,19 +20,20 @@ import java.util import java.util.concurrent.locks.ReentrantReadWriteLock import com.typesafe.scalalogging.Logger -import kafka.common.{NotificationHandler, ZkNodeChangeNotificationListener} +import kafka.api.KAFKA_2_0_IV1 import kafka.network.RequestChannel.Session -import kafka.security.auth.SimpleAclAuthorizer.VersionedAcls +import kafka.security.auth.SimpleAclAuthorizer.{NoAcls, VersionedAcls} import kafka.server.KafkaConfig import kafka.utils.CoreUtils.{inReadLock, inWriteLock} import kafka.utils._ -import kafka.zk.{AclChangeNotificationSequenceZNode, KafkaZkClient, ZkAclStore} -import org.apache.kafka.common.resource.{ResourceFilter, ResourceNameType => JResourceNameType} +import kafka.zk.{AclChangeNotificationHandler, AclChangeSubscription, KafkaZkClient, ZkAclChangeStore, ZkAclStore, ZkVersion} +import org.apache.kafka.common.errors.UnsupportedVersionException +import org.apache.kafka.common.resource.PatternType import org.apache.kafka.common.security.auth.KafkaPrincipal import org.apache.kafka.common.utils.{SecurityUtils, Time} import scala.collection.JavaConverters._ -import scala.util.Random +import scala.util.{Failure, Random, Success, Try} object SimpleAclAuthorizer { //optional override zookeeper cluster configuration where acls will be stored, if not specified acls will be stored in @@ -47,7 +48,10 @@ object SimpleAclAuthorizer { //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" - case class VersionedAcls(acls: Set[Acl], zkVersion: Int) + case class VersionedAcls(acls: Set[Acl], zkVersion: Int) { + def exists: Boolean = zkVersion != ZkVersion.UnknownVersion + } + val NoAcls = VersionedAcls(Set.empty, ZkVersion.UnknownVersion) } class SimpleAclAuthorizer extends Authorizer with Logging { @@ -55,7 +59,8 @@ class SimpleAclAuthorizer extends Authorizer with Logging { private var superUsers = Set.empty[KafkaPrincipal] private var shouldAllowEveryoneIfNoAclIsFound = false private var zkClient: KafkaZkClient = _ - private var aclChangeListeners: Seq[ZkNodeChangeNotificationListener] = List() + private var aclChangeListeners: Iterable[AclChangeSubscription] = Iterable.empty + private var extendedAclSupport: Boolean = _ @volatile private var aclCache = new scala.collection.immutable.TreeMap[Resource, VersionedAcls]()(ResourceOrdering) @@ -96,17 +101,26 @@ class SimpleAclAuthorizer extends Authorizer with Logging { zkMaxInFlightRequests, time, "kafka.security", "SimpleAclAuthorizer") zkClient.createAclPaths() - loadCache() + extendedAclSupport = kafkaConfig.interBrokerProtocolVersion >= KAFKA_2_0_IV1 + // Start change listeners first and then populate the cache so that there is no timing window + // between loading cache and processing change notifications. startZkChangeListeners() + loadCache() } override def authorize(session: Session, operation: Operation, resource: Resource): Boolean = { - if (resource.resourceNameType != Literal) { - throw new IllegalArgumentException("Only literal resources are supported. Got: " + resource.resourceNameType) + if (resource.patternType != PatternType.LITERAL) { + throw new IllegalArgumentException("Only literal resources are supported. Got: " + resource.patternType) } - val principal = session.principal + // ensure we compare identical classes + val sessionPrincipal = session.principal + val principal = if (classOf[KafkaPrincipal] != sessionPrincipal.getClass) + new KafkaPrincipal(sessionPrincipal.getPrincipalType, sessionPrincipal.getName) + else + sessionPrincipal + val host = session.clientAddress.getHostAddress val acls = getMatchingAcls(resource.resourceType, resource.name) @@ -161,6 +175,11 @@ class SimpleAclAuthorizer extends Authorizer with Logging { override def addAcls(acls: Set[Acl], resource: Resource) { if (acls != null && acls.nonEmpty) { + if (!extendedAclSupport && resource.patternType == PatternType.PREFIXED) { + throw new UnsupportedVersionException(s"Adding ACLs on prefixed resource patterns requires " + + s"${KafkaConfig.InterBrokerProtocolVersionProp} of $KAFKA_2_0_IV1 or greater") + } + inWriteLock(lock) { updateResourceAcls(resource) { currentAcls => currentAcls ++ acls @@ -180,7 +199,7 @@ class SimpleAclAuthorizer extends Authorizer with Logging { override def removeAcls(resource: Resource): Boolean = { inWriteLock(lock) { val result = zkClient.deleteResource(resource) - updateCache(resource, VersionedAcls(Set(), 0)) + updateCache(resource, NoAcls) updateAclChangedFlag(resource) result } @@ -203,18 +222,18 @@ class SimpleAclAuthorizer extends Authorizer with Logging { } def getMatchingAcls(resourceType: ResourceType, resourceName: String): Set[Acl] = { - val filter = new ResourceFilter(resourceType.toJava, resourceName, JResourceNameType.ANY) - inReadLock(lock) { - val wildcard = aclCache.get(Resource(resourceType, Acl.WildCardResource, Literal)) + val wildcard = aclCache.get(Resource(resourceType, Acl.WildCardResource, PatternType.LITERAL)) .map(_.acls) .getOrElse(Set.empty[Acl]) - val literal = aclCache.get(Resource(resourceType, resourceName, Literal)) + val literal = aclCache.get(Resource(resourceType, resourceName, PatternType.LITERAL)) .map(_.acls) .getOrElse(Set.empty[Acl]) - val prefixed = aclCache.range(Resource(resourceType, resourceName, Prefixed), Resource(resourceType, resourceName.substring(0, 1), Prefixed)) + val prefixed = aclCache + .from(Resource(resourceType, resourceName, PatternType.PREFIXED)) + .to(Resource(resourceType, resourceName.take(1), PatternType.PREFIXED)) .filterKeys(resource => resourceName.startsWith(resource.name)) .flatMap { case (resource, versionedAcls) => versionedAcls.acls } .toSet @@ -225,7 +244,7 @@ class SimpleAclAuthorizer extends Authorizer with Logging { override def getAcls(): Map[Resource, Set[Acl]] = { inReadLock(lock) { - aclCache.mapValues(_.acls).toMap + aclCache.mapValues(_.acls) } } @@ -237,27 +256,28 @@ class SimpleAclAuthorizer extends Authorizer with Logging { private def loadCache() { inWriteLock(lock) { ZkAclStore.stores.foreach(store => { - val resourceTypes = zkClient.getResourceTypes(store.nameType) + val resourceTypes = zkClient.getResourceTypes(store.patternType) for (rType <- resourceTypes) { - val resourceType = ResourceType.fromString(rType) - val resourceNames = zkClient.getResourceNames(store.nameType, resourceType) - for (resourceName <- resourceNames) { - val versionedAcls = getAclsFromZk(new Resource(resourceType, resourceName, store.nameType)) - updateCache(new Resource(resourceType, resourceName, store.nameType), versionedAcls) + val resourceType = Try(ResourceType.fromString(rType)) + resourceType match { + case Success(resourceTypeObj) => { + val resourceNames = zkClient.getResourceNames(store.patternType, resourceTypeObj) + for (resourceName <- resourceNames) { + val resource = new Resource(resourceTypeObj, resourceName, store.patternType) + val versionedAcls = getAclsFromZk(resource) + updateCache(resource, versionedAcls) + } + } + case Failure(f) => warn(s"Ignoring unknown ResourceType: $rType") } } }) } } - private def startZkChangeListeners(): Unit = { - aclChangeListeners = ZkAclStore.stores.map(store => { - val aclChangeListener = new ZkNodeChangeNotificationListener( - zkClient, store.aclChangePath, AclChangeNotificationSequenceZNode.SequenceNumberPrefix, new AclChangedNotificationHandler(store)) - - aclChangeListener.init() - aclChangeListener - }) + private[auth] def startZkChangeListeners(): Unit = { + aclChangeListeners = ZkAclChangeStore.stores + .map(store => store.createListener(AclChangedNotificationHandler, zkClient)) } private def logAuditMessage(principal: KafkaPrincipal, authorized: Boolean, operation: Operation, resource: Resource, host: String) { @@ -293,7 +313,10 @@ class SimpleAclAuthorizer extends Authorizer with Logging { val newAcls = getNewAcls(currentVersionedAcls.acls) val (updateSucceeded, updateVersion) = if (newAcls.nonEmpty) { - zkClient.conditionalSetOrCreateAclsForResource(resource, newAcls, currentVersionedAcls.zkVersion) + if (currentVersionedAcls.exists) + zkClient.conditionalSetAclsForResource(resource, newAcls, currentVersionedAcls.zkVersion) + else + zkClient.createAclsForResourceIfNotExists(resource, newAcls) } else { trace(s"Deleting path for $resource because it had no ACLs remaining") (zkClient.conditionalDelete(resource, currentVersionedAcls.zkVersion), 0) @@ -342,17 +365,15 @@ class SimpleAclAuthorizer extends Authorizer with Logging { } private def updateAclChangedFlag(resource: Resource) { - zkClient.createAclChangeNotification(resource) + zkClient.createAclChangeNotification(resource) } private def backoffTime = { retryBackoffMs + Random.nextInt(retryBackoffJitterMs) } - class AclChangedNotificationHandler(store: ZkAclStore) extends NotificationHandler { - override def processNotification(notificationMessage: Array[Byte]) { - val resource: Resource = store.decode(notificationMessage) - + object AclChangedNotificationHandler extends AclChangeNotificationHandler { + override def processNotification(resource: Resource) { inWriteLock(lock) { val versionedAcls = getAclsFromZk(resource) updateCache(resource, versionedAcls) @@ -360,7 +381,7 @@ class SimpleAclAuthorizer extends Authorizer with Logging { } } - // Orders by resource type, then resource name type and finally reverse ordering by name. + // Orders by resource type, then resource pattern type and finally reverse ordering by name. private object ResourceOrdering extends Ordering[Resource] { def compare(a: Resource, b: Resource): Int = { @@ -368,7 +389,7 @@ class SimpleAclAuthorizer extends Authorizer with Logging { if (rt != 0) rt else { - val rnt = a.resourceNameType compare b.resourceNameType + val rnt = a.patternType compareTo b.patternType if (rnt != 0) rnt else diff --git a/core/src/main/scala/kafka/server/AbstractFetcherThread.scala b/core/src/main/scala/kafka/server/AbstractFetcherThread.scala index f27dbfe07eb37..9588356b92b91 100755 --- a/core/src/main/scala/kafka/server/AbstractFetcherThread.scala +++ b/core/src/main/scala/kafka/server/AbstractFetcherThread.scala @@ -19,11 +19,11 @@ package kafka.server import java.util.concurrent.locks.ReentrantLock -import kafka.cluster.{Replica, BrokerEndPoint} +import kafka.cluster.{BrokerEndPoint, Replica} import kafka.utils.{DelayedItem, Pool, ShutdownableThread} import org.apache.kafka.common.errors.{CorruptRecordException, KafkaStorageException} import org.apache.kafka.common.requests.EpochEndOffset._ -import kafka.common.{ClientIdAndBroker, KafkaException} +import kafka.common.ClientIdAndBroker import kafka.metrics.KafkaMetricsGroup import kafka.utils.CoreUtils.inLock import org.apache.kafka.common.protocol.Errors @@ -35,8 +35,8 @@ import java.util.concurrent.TimeUnit import java.util.concurrent.atomic.AtomicLong import com.yammer.metrics.core.Gauge -import org.apache.kafka.common.TopicPartition -import org.apache.kafka.common.internals.{FatalExitError, PartitionStates} +import org.apache.kafka.common.{KafkaException, TopicPartition} +import org.apache.kafka.common.internals.PartitionStates import org.apache.kafka.common.record.MemoryRecords import org.apache.kafka.common.requests.EpochEndOffset @@ -75,7 +75,7 @@ abstract class AbstractFetcherThread(name: String, // deal with partitions with errors, potentially due to leadership changes protected def handlePartitionsWithErrors(partitions: Iterable[TopicPartition]) - protected def buildLeaderEpochRequest(allPartitions: Seq[(TopicPartition, PartitionFetchState)]): ResultWithPartitions[Map[TopicPartition, Int]] + protected def buildLeaderEpochRequest(allPartitions: Seq[(TopicPartition, PartitionFetchState)]): (Map[TopicPartition, Int], Set[TopicPartition]) protected def fetchEpochsFromLeader(partitions: Map[TopicPartition, Int]): Map[TopicPartition, EpochEndOffset] @@ -124,15 +124,22 @@ abstract class AbstractFetcherThread(name: String, * occur during truncation. */ def maybeTruncate(): Unit = { - val ResultWithPartitions(epochRequests, partitionsWithError) = inLock(partitionMapLock) { buildLeaderEpochRequest(states) } - handlePartitionsWithErrors(partitionsWithError) + val (partitionsWithEpochs, partitionsWithoutEpochs) = inLock(partitionMapLock) { buildLeaderEpochRequest(states) } + val epochEndOffsets = mutable.Map[TopicPartition, EpochEndOffset]() - if (epochRequests.nonEmpty) { - val fetchedEpochs = fetchEpochsFromLeader(epochRequests) + // If the latest epoch is not available, then we are likely on an older format and should + // use the high watermark for truncation + for (tp <- partitionsWithoutEpochs) + epochEndOffsets.put(tp, new EpochEndOffset(Errors.NONE, UNDEFINED_EPOCH, UNDEFINED_EPOCH_OFFSET)) + + if (partitionsWithEpochs.nonEmpty) + epochEndOffsets ++= fetchEpochsFromLeader(partitionsWithEpochs) + + if (epochEndOffsets.nonEmpty) { //Ensure we hold a lock during truncation. inLock(partitionMapLock) { //Check no leadership changes happened whilst we were unlocked, fetching epochs - val leaderEpochs = fetchedEpochs.filter { case (tp, _) => partitionStates.contains(tp) } + val leaderEpochs = epochEndOffsets.filter { case (tp, _) => partitionStates.contains(tp) } val ResultWithPartitions(fetchOffsets, partitionsWithError) = maybeTruncate(leaderEpochs) handlePartitionsWithErrors(partitionsWithError) updateFetchOffsetAndMaybeMarkTruncationComplete(fetchOffsets) @@ -214,7 +221,6 @@ abstract class AbstractFetcherThread(name: String, info(s"Current offset ${currentPartitionFetchState.fetchOffset} for partition $topicPartition is " + s"out of range, which typically implies a leader change. Reset fetch offset to $newOffset") } catch { - case e: FatalExitError => throw e case e: Throwable => error(s"Error getting offset for partition $topicPartition", e) partitionsWithError += topicPartition @@ -339,7 +345,7 @@ abstract class AbstractFetcherThread(name: String, } else { // get (leader epoch, end offset) pair that corresponds to the largest leader epoch // less than or equal to the requested epoch. - val (followerEpoch, followerEndOffset) = replica.epochs.get.endOffsetFor(leaderEpochOffset.leaderEpoch) + val (followerEpoch, followerEndOffset) = replica.endOffsetFor(leaderEpochOffset.leaderEpoch) if (followerEndOffset == UNDEFINED_EPOCH_OFFSET) { // This can happen if the follower was not tracking leader epochs at that point (before the // upgrade, or if this broker is new). Since the leader replied with epoch < diff --git a/core/src/main/scala/kafka/server/AdminManager.scala b/core/src/main/scala/kafka/server/AdminManager.scala index 01457a197319a..e9598e365ab0d 100644 --- a/core/src/main/scala/kafka/server/AdminManager.scala +++ b/core/src/main/scala/kafka/server/AdminManager.scala @@ -33,7 +33,7 @@ import org.apache.kafka.common.network.ListenerName import org.apache.kafka.common.protocol.Errors import org.apache.kafka.common.requests.CreateTopicsRequest._ import org.apache.kafka.common.requests.DescribeConfigsResponse.ConfigSource -import org.apache.kafka.common.requests.{AlterConfigsRequest, ApiError, DescribeConfigsResponse, Resource, ResourceType} +import org.apache.kafka.common.requests.{AlterConfigsRequest, ApiError, DescribeConfigsResponse} import org.apache.kafka.server.policy.{AlterConfigPolicy, CreateTopicPolicy} import org.apache.kafka.server.policy.CreateTopicPolicy.RequestMetadata @@ -281,7 +281,7 @@ class AdminManager(val config: KafkaConfig, } } - def describeConfigs(resourceToConfigNames: Map[Resource, Option[Set[String]]], includeSynonyms: Boolean): Map[Resource, DescribeConfigsResponse.Config] = { + def describeConfigs(resourceToConfigNames: Map[ConfigResource, Option[Set[String]]], includeSynonyms: Boolean): Map[ConfigResource, DescribeConfigsResponse.Config] = { resourceToConfigNames.map { case (resource, configNames) => def allConfigs(config: AbstractConfig) = { @@ -301,7 +301,7 @@ class AdminManager(val config: KafkaConfig, try { val resourceConfig = resource.`type` match { - case ResourceType.TOPIC => + case ConfigResource.Type.TOPIC => val topic = resource.name Topic.validate(topic) if (metadataCache.contains(topic)) { @@ -313,7 +313,7 @@ class AdminManager(val config: KafkaConfig, new DescribeConfigsResponse.Config(new ApiError(Errors.UNKNOWN_TOPIC_OR_PARTITION, null), Collections.emptyList[DescribeConfigsResponse.ConfigEntry]) } - case ResourceType.BROKER => + case ConfigResource.Type.BROKER => if (resource.name == null || resource.name.isEmpty) createResponseConfig(config.dynamicConfig.currentDynamicDefaultConfigs, createBrokerConfigEntry(perBrokerConfig = false, includeSynonyms)) @@ -339,7 +339,7 @@ class AdminManager(val config: KafkaConfig, }.toMap } - def alterConfigs(configs: Map[Resource, AlterConfigsRequest.Config], validateOnly: Boolean): Map[Resource, ApiError] = { + def alterConfigs(configs: Map[ConfigResource, AlterConfigsRequest.Config], validateOnly: Boolean): Map[ConfigResource, ApiError] = { configs.map { case (resource, config) => def validateConfigPolicy(resourceType: ConfigResource.Type): Unit = { @@ -353,7 +353,7 @@ class AdminManager(val config: KafkaConfig, } try { resource.`type` match { - case ResourceType.TOPIC => + case ConfigResource.Type.TOPIC => val topic = resource.name val properties = new Properties @@ -368,7 +368,7 @@ class AdminManager(val config: KafkaConfig, resource -> ApiError.NONE - case ResourceType.BROKER => + case ConfigResource.Type.BROKER => val brokerId = if (resource.name == null || resource.name.isEmpty) None else { diff --git a/core/src/main/scala/kafka/server/DynamicBrokerConfig.scala b/core/src/main/scala/kafka/server/DynamicBrokerConfig.scala index 4225cdb1a4058..4bd0be7392d71 100755 --- a/core/src/main/scala/kafka/server/DynamicBrokerConfig.scala +++ b/core/src/main/scala/kafka/server/DynamicBrokerConfig.scala @@ -115,6 +115,43 @@ object DynamicBrokerConfig { } } + def validateConfigs(props: Properties, perBrokerConfig: Boolean): Unit = { + def checkInvalidProps(invalidPropNames: Set[String], errorMessage: String): Unit = { + if (invalidPropNames.nonEmpty) + throw new ConfigException(s"$errorMessage: $invalidPropNames") + } + checkInvalidProps(nonDynamicConfigs(props), "Cannot update these configs dynamically") + checkInvalidProps(securityConfigsWithoutListenerPrefix(props), + "These security configs can be dynamically updated only per-listener using the listener prefix") + validateConfigTypes(props) + if (!perBrokerConfig) { + checkInvalidProps(perBrokerConfigs(props), + "Cannot update these configs at default cluster level, broker id must be specified") + } + } + + private def perBrokerConfigs(props: Properties): Set[String] = { + val configNames = props.asScala.keySet + configNames.intersect(PerBrokerConfigs) ++ configNames.filter(ListenerConfigRegex.findFirstIn(_).nonEmpty) + } + + private def nonDynamicConfigs(props: Properties): Set[String] = { + props.asScala.keySet.intersect(DynamicConfig.Broker.nonDynamicProps) + } + + private def securityConfigsWithoutListenerPrefix(props: Properties): Set[String] = { + DynamicSecurityConfigs.filter(props.containsKey) + } + + private def validateConfigTypes(props: Properties): Unit = { + val baseProps = new Properties + props.asScala.foreach { + case (ListenerConfigRegex(baseName), v) => baseProps.put(baseName, v) + case (k, v) => baseProps.put(k, v) + } + DynamicConfig.Broker.validate(baseProps) + } + private[server] def addDynamicConfigs(configDef: ConfigDef): Unit = { KafkaConfig.configKeys.filterKeys(AllDynamicConfigs.contains).values.foreach { config => configDef.define(config.name, config.`type`, config.defaultValue, config.validator, @@ -152,6 +189,18 @@ class DynamicBrokerConfig(private val kafkaConfig: KafkaConfig) extends Logging updateBrokerConfig(kafkaConfig.brokerId, brokerConfig) } + /** + * Clear all cached values. This is used to clear state on broker shutdown to avoid + * exceptions in tests when broker is restarted. These fields are re-initialized when + * broker starts up. + */ + private[server] def clear(): Unit = { + dynamicBrokerConfigs.clear() + dynamicDefaultConfigs.clear() + reconfigurables.clear() + brokerReconfigurables.clear() + } + def addReconfigurables(kafkaServer: KafkaServer): Unit = { addBrokerReconfigurable(new DynamicThreadPool(kafkaServer)) if (kafkaServer.logManager.cleaner != null) @@ -298,57 +347,26 @@ class DynamicBrokerConfig(private val kafkaConfig: KafkaConfig) extends Logging decoded.foreach { value => props.put(configName, passwordEncoder.encode(new Password(value))) } } } - adminZkClient.changeBrokerConfig(Seq(kafkaConfig.brokerId), props) + adminZkClient.changeBrokerConfig(Some(kafkaConfig.brokerId), props) } } props } private[server] def validate(props: Properties, perBrokerConfig: Boolean): Unit = CoreUtils.inReadLock(lock) { - def checkInvalidProps(invalidPropNames: Set[String], errorMessage: String): Unit = { - if (invalidPropNames.nonEmpty) - throw new ConfigException(s"$errorMessage: $invalidPropNames") - } - checkInvalidProps(nonDynamicConfigs(props), "Cannot update these configs dynamically") - checkInvalidProps(securityConfigsWithoutListenerPrefix(props), - "These security configs can be dynamically updated only per-listener using the listener prefix") - validateConfigTypes(props) + validateConfigs(props, perBrokerConfig) val newProps = mutable.Map[String, String]() newProps ++= staticBrokerConfigs if (perBrokerConfig) { overrideProps(newProps, dynamicDefaultConfigs) overrideProps(newProps, props.asScala) } else { - checkInvalidProps(perBrokerConfigs(props), - "Cannot update these configs at default cluster level, broker id must be specified") overrideProps(newProps, props.asScala) overrideProps(newProps, dynamicBrokerConfigs) } processReconfiguration(newProps, validateOnly = true) } - private def perBrokerConfigs(props: Properties): Set[String] = { - val configNames = props.asScala.keySet - configNames.intersect(PerBrokerConfigs) ++ configNames.filter(ListenerConfigRegex.findFirstIn(_).nonEmpty) - } - - private def nonDynamicConfigs(props: Properties): Set[String] = { - props.asScala.keySet.intersect(DynamicConfig.Broker.nonDynamicProps) - } - - private def securityConfigsWithoutListenerPrefix(props: Properties): Set[String] = { - DynamicSecurityConfigs.filter(props.containsKey) - } - - private def validateConfigTypes(props: Properties): Unit = { - val baseProps = new Properties - props.asScala.foreach { - case (ListenerConfigRegex(baseName), v) => baseProps.put(baseName, v) - case (k, v) => baseProps.put(k, v) - } - DynamicConfig.Broker.validate(baseProps) - } - private def removeInvalidConfigs(props: Properties, perBrokerConfig: Boolean): Unit = { try { validateConfigTypes(props) diff --git a/core/src/main/scala/kafka/server/FetchSession.scala b/core/src/main/scala/kafka/server/FetchSession.scala index 7a47780a1352d..64bc773f0bbb8 100644 --- a/core/src/main/scala/kafka/server/FetchSession.scala +++ b/core/src/main/scala/kafka/server/FetchSession.scala @@ -92,22 +92,22 @@ class CachedPartition(val topic: String, this(topic, partition, -1, -1, -1, -1, -1) def this(part: TopicPartition) = - this(part.topic(), part.partition()) + this(part.topic, part.partition) def this(part: TopicPartition, reqData: FetchRequest.PartitionData) = - this(part.topic(), part.partition(), + this(part.topic, part.partition, reqData.maxBytes, reqData.fetchOffset, -1, reqData.logStartOffset, -1) def this(part: TopicPartition, reqData: FetchRequest.PartitionData, respData: FetchResponse.PartitionData[Records]) = - this(part.topic(), part.partition(), + this(part.topic, part.partition, reqData.maxBytes, reqData.fetchOffset, respData.highWatermark, reqData.logStartOffset, respData.logStartOffset) - def topicPartition() = new TopicPartition(topic, partition) + def topicPartition = new TopicPartition(topic, partition) - def reqData() = new FetchRequest.PartitionData(fetchOffset, fetcherLogStartOffset, maxBytes) + def reqData = new FetchRequest.PartitionData(fetchOffset, fetcherLogStartOffset, maxBytes) def updateRequestParams(reqData: FetchRequest.PartitionData): Unit = { // Update our cached request parameters. @@ -129,7 +129,7 @@ class CachedPartition(val topic: String, def maybeUpdateResponseData(respData: FetchResponse.PartitionData[Records], updateResponseData: Boolean): Boolean = { // Check the response data. var mustRespond = false - if ((respData.records != null) && (respData.records.sizeInBytes() > 0)) { + if ((respData.records != null) && (respData.records.sizeInBytes > 0)) { // Partitions with new data are always included in the response. mustRespond = true } @@ -143,7 +143,7 @@ class CachedPartition(val topic: String, if (updateResponseData) localLogStartOffset = respData.logStartOffset } - if (respData.error.code() != 0) { + if (respData.error.code != 0) { // Partitions with errors are always included in the response. // We also set the cached highWatermark to an invalid offset, -1. // This ensures that when the error goes away, we re-send the partition. @@ -154,7 +154,7 @@ class CachedPartition(val topic: String, mustRespond } - override def hashCode() = (31 * partition) + topic.hashCode + override def hashCode = (31 * partition) + topic.hashCode def canEqual(that: Any) = that.isInstanceOf[CachedPartition] @@ -166,7 +166,7 @@ class CachedPartition(val topic: String, case _ => false } - override def toString() = synchronized { + override def toString = synchronized { "CachedPartition(topic=" + topic + ", partition=" + partition + ", maxBytes=" + maxBytes + @@ -203,23 +203,23 @@ case class FetchSession(val id: Int, // If this is -1, the Session is not in the cache. var cachedSize = -1 - def size(): Int = synchronized { - partitionMap.size() + def size: Int = synchronized { + partitionMap.size } - def isEmpty(): Boolean = synchronized { + def isEmpty: Boolean = synchronized { partitionMap.isEmpty } - def lastUsedKey(): LastUsedKey = synchronized { + def lastUsedKey: LastUsedKey = synchronized { LastUsedKey(lastUsedMs, id) } - def evictableKey(): EvictableKey = synchronized { + def evictableKey: EvictableKey = synchronized { EvictableKey(privileged, cachedSize, id) } - def metadata(): JFetchMetadata = synchronized { new JFetchMetadata(id, epoch) } + def metadata: JFetchMetadata = synchronized { new JFetchMetadata(id, epoch) } def getFetchOffset(topicPartition: TopicPartition): Option[Long] = synchronized { Option(partitionMap.find(new CachedPartition(topicPartition))).map(_.fetchOffset) @@ -234,7 +234,7 @@ case class FetchSession(val id: Int, val added = new TL val updated = new TL val removed = new TL - fetchData.entrySet().iterator().asScala.foreach(entry => { + fetchData.entrySet.iterator.asScala.foreach(entry => { val topicPart = entry.getKey val reqData = entry.getValue val newCachedPart = new CachedPartition(topicPart, reqData) @@ -247,18 +247,18 @@ case class FetchSession(val id: Int, updated.add(topicPart) } }) - toForget.iterator().asScala.foreach(p => { - if (partitionMap.remove(new CachedPartition(p.topic(), p.partition()))) { + toForget.iterator.asScala.foreach(p => { + if (partitionMap.remove(new CachedPartition(p.topic, p.partition))) { removed.add(p) } }) (added, updated, removed) } - override def toString(): String = synchronized { + override def toString: String = synchronized { "FetchSession(id=" + id + ", privileged=" + privileged + - ", partitionMap.size=" + partitionMap.size() + + ", partitionMap.size=" + partitionMap.size + ", creationMs=" + creationMs + ", creationMs=" + lastUsedMs + ", epoch=" + epoch + ")" @@ -290,6 +290,12 @@ trait FetchContext extends Logging { def partitionsToLogString(partitions: util.Collection[TopicPartition]): String = FetchSession.partitionsToLogString(partitions, isTraceEnabled) + + /** + * Return an empty throttled response due to quota violation. + */ + def getThrottledResponse(throttleTimeMs: Int): FetchResponse[Records] = + new FetchResponse(Errors.NONE, new FetchSession.RESP_MAP, throttleTimeMs, INVALID_SESSION_ID) } /** @@ -302,7 +308,7 @@ class SessionErrorContext(val error: Errors, override def foreachPartition(fun: (TopicPartition, FetchRequest.PartitionData) => Unit): Unit = {} override def getResponseSize(updates: FetchSession.RESP_MAP, versionId: Short): Int = { - FetchResponse.sizeOf(versionId, (new FetchSession.RESP_MAP).entrySet().iterator()) + FetchResponse.sizeOf(versionId, (new FetchSession.RESP_MAP).entrySet.iterator) } // Because of the fetch session error, we don't know what partitions were supposed to be in this request. @@ -322,15 +328,15 @@ class SessionlessFetchContext(val fetchData: util.Map[TopicPartition, FetchReque Option(fetchData.get(part)).map(_.fetchOffset) override def foreachPartition(fun: (TopicPartition, FetchRequest.PartitionData) => Unit): Unit = { - fetchData.entrySet().asScala.foreach(entry => fun(entry.getKey, entry.getValue)) + fetchData.entrySet.asScala.foreach(entry => fun(entry.getKey, entry.getValue)) } override def getResponseSize(updates: FetchSession.RESP_MAP, versionId: Short): Int = { - FetchResponse.sizeOf(versionId, updates.entrySet().iterator()) + FetchResponse.sizeOf(versionId, updates.entrySet.iterator) } override def updateAndGenerateResponseData(updates: FetchSession.RESP_MAP): FetchResponse[Records] = { - debug(s"Sessionless fetch context returning ${partitionsToLogString(updates.keySet())}") + debug(s"Sessionless fetch context returning ${partitionsToLogString(updates.keySet)}") new FetchResponse(Errors.NONE, updates, 0, INVALID_SESSION_ID) } } @@ -353,17 +359,17 @@ class FullFetchContext(private val time: Time, Option(fetchData.get(part)).map(_.fetchOffset) override def foreachPartition(fun: (TopicPartition, FetchRequest.PartitionData) => Unit): Unit = { - fetchData.entrySet().asScala.foreach(entry => fun(entry.getKey, entry.getValue)) + fetchData.entrySet.asScala.foreach(entry => fun(entry.getKey, entry.getValue)) } override def getResponseSize(updates: FetchSession.RESP_MAP, versionId: Short): Int = { - FetchResponse.sizeOf(versionId, updates.entrySet().iterator()) + FetchResponse.sizeOf(versionId, updates.entrySet.iterator) } override def updateAndGenerateResponseData(updates: FetchSession.RESP_MAP): FetchResponse[Records] = { - def createNewSession(): FetchSession.CACHE_MAP = { - val cachedPartitions = new FetchSession.CACHE_MAP(updates.size()) - updates.entrySet().asScala.foreach(entry => { + def createNewSession: FetchSession.CACHE_MAP = { + val cachedPartitions = new FetchSession.CACHE_MAP(updates.size) + updates.entrySet.asScala.foreach(entry => { val part = entry.getKey val respData = entry.getValue val reqData = fetchData.get(part) @@ -372,9 +378,9 @@ class FullFetchContext(private val time: Time, cachedPartitions } val responseSessionId = cache.maybeCreateSession(time.milliseconds(), isFromFollower, - updates.size(), createNewSession) + updates.size, () => createNewSession) debug(s"Full fetch context with session id $responseSessionId returning " + - s"${partitionsToLogString(updates.keySet())}") + s"${partitionsToLogString(updates.keySet)}") new FetchResponse(Errors.NONE, updates, 0, responseSessionId) } } @@ -395,8 +401,8 @@ class IncrementalFetchContext(private val time: Time, override def foreachPartition(fun: (TopicPartition, FetchRequest.PartitionData) => Unit): Unit = { // Take the session lock and iterate over all the cached partitions. session.synchronized { - session.partitionMap.iterator().asScala.foreach(part => { - fun(new TopicPartition(part.topic, part.partition), part.reqData()) + session.partitionMap.iterator.asScala.foreach(part => { + fun(new TopicPartition(part.topic, part.partition), part.reqData) }) } } @@ -410,7 +416,7 @@ class IncrementalFetchContext(private val time: Time, var nextElement: util.Map.Entry[TopicPartition, FetchResponse.PartitionData[Records]] = null override def hasNext: Boolean = { - while ((nextElement == null) && iter.hasNext()) { + while ((nextElement == null) && iter.hasNext) { val element = iter.next() val topicPart = element.getKey val respData = element.getValue @@ -432,23 +438,23 @@ class IncrementalFetchContext(private val time: Time, } override def next(): util.Map.Entry[TopicPartition, FetchResponse.PartitionData[Records]] = { - if (!hasNext()) throw new NoSuchElementException() + if (!hasNext) throw new NoSuchElementException val element = nextElement nextElement = null element } - override def remove() = throw new UnsupportedOperationException() + override def remove() = throw new UnsupportedOperationException } override def getResponseSize(updates: FetchSession.RESP_MAP, versionId: Short): Int = { session.synchronized { - val expectedEpoch = JFetchMetadata.nextEpoch(reqMetadata.epoch()) + val expectedEpoch = JFetchMetadata.nextEpoch(reqMetadata.epoch) if (session.epoch != expectedEpoch) { - FetchResponse.sizeOf(versionId, (new FetchSession.RESP_MAP).entrySet().iterator()) + FetchResponse.sizeOf(versionId, (new FetchSession.RESP_MAP).entrySet.iterator) } else { // Pass the partition iterator which updates neither the fetch context nor the partition map. - FetchResponse.sizeOf(versionId, new PartitionIterator(updates.entrySet().iterator(), false)) + FetchResponse.sizeOf(versionId, new PartitionIterator(updates.entrySet.iterator, false)) } } } @@ -457,23 +463,38 @@ class IncrementalFetchContext(private val time: Time, session.synchronized { // Check to make sure that the session epoch didn't change in between // creating this fetch context and generating this response. - val expectedEpoch = JFetchMetadata.nextEpoch(reqMetadata.epoch()) + val expectedEpoch = JFetchMetadata.nextEpoch(reqMetadata.epoch) if (session.epoch != expectedEpoch) { info(s"Incremental fetch session ${session.id} expected epoch $expectedEpoch, but " + s"got ${session.epoch}. Possible duplicate request.") new FetchResponse(Errors.INVALID_FETCH_SESSION_EPOCH, new FetchSession.RESP_MAP, 0, session.id) } else { // Iterate over the update list using PartitionIterator. This will prune updates which don't need to be sent - val partitionIter = new PartitionIterator(updates.entrySet().iterator(), true) - while (partitionIter.hasNext()) { + val partitionIter = new PartitionIterator(updates.entrySet.iterator, true) + while (partitionIter.hasNext) { partitionIter.next() } debug(s"Incremental fetch context with session id ${session.id} returning " + - s"${partitionsToLogString(updates.keySet())}") + s"${partitionsToLogString(updates.keySet)}") new FetchResponse(Errors.NONE, updates, 0, session.id) } } } + + override def getThrottledResponse(throttleTimeMs: Int): FetchResponse[Records] = { + session.synchronized { + // Check to make sure that the session epoch didn't change in between + // creating this fetch context and generating this response. + val expectedEpoch = JFetchMetadata.nextEpoch(reqMetadata.epoch) + if (session.epoch != expectedEpoch) { + info(s"Incremental fetch session ${session.id} expected epoch $expectedEpoch, but " + + s"got ${session.epoch}. Possible duplicate request.") + new FetchResponse(Errors.INVALID_FETCH_SESSION_EPOCH, new FetchSession.RESP_MAP, throttleTimeMs, session.id) + } else { + new FetchResponse(Errors.NONE, new FetchSession.RESP_MAP, throttleTimeMs, session.id) + } + } + } } case class LastUsedKey(val lastUsedMs: Long, @@ -549,14 +570,14 @@ class FetchSessionCache(private val maxEntries: Int, /** * Get the number of entries currently in the fetch session cache. */ - def size(): Int = synchronized { + def size: Int = synchronized { sessions.size } /** * Get the total number of cached partitions. */ - def totalPartitions(): Long = synchronized { + def totalPartitions: Long = synchronized { numPartitions } @@ -593,7 +614,7 @@ class FetchSessionCache(private val maxEntries: Int, val partitionMap = createPartitions() val session = new FetchSession(newSessionId(), privileged, partitionMap, now, now, JFetchMetadata.nextEpoch(INITIAL_EPOCH)) - debug(s"Created fetch session ${session.toString()}") + debug(s"Created fetch session ${session.toString}") sessions.put(session.id, session) touch(session, now) session.id @@ -618,12 +639,12 @@ class FetchSessionCache(private val maxEntries: Int, */ def tryEvict(privileged: Boolean, key: EvictableKey, now: Long): Boolean = synchronized { // Try to evict an entry which is stale. - val lastUsedEntry = lastUsed.firstEntry() + val lastUsedEntry = lastUsed.firstEntry if (lastUsedEntry == null) { trace("There are no cache entries to evict.") false - } else if (now - lastUsedEntry.getKey().lastUsedMs > evictionMs) { - val session = lastUsedEntry.getValue() + } else if (now - lastUsedEntry.getKey.lastUsedMs > evictionMs) { + val session = lastUsedEntry.getValue trace(s"Evicting stale FetchSession ${session.id}.") remove(session) evictionsMeter.mark() @@ -632,16 +653,16 @@ class FetchSessionCache(private val maxEntries: Int, // If there are no stale entries, check the first evictable entry. // If it is less valuable than our proposed entry, evict it. val map = if (privileged) evictableByPrivileged else evictableByAll - val evictableEntry = map.firstEntry() + val evictableEntry = map.firstEntry if (evictableEntry == null) { trace("No evictable entries found.") false - } else if (key.compareTo(evictableEntry.getKey()) < 0) { - trace(s"Can't evict ${evictableEntry.getKey()} with ${key.toString}") + } else if (key.compareTo(evictableEntry.getKey) < 0) { + trace(s"Can't evict ${evictableEntry.getKey} with ${key.toString}") false } else { - trace(s"Evicting ${evictableEntry.getKey()} with ${key.toString}.") - remove(evictableEntry.getValue()) + trace(s"Evicting ${evictableEntry.getKey} with ${key.toString}.") + remove(evictableEntry.getValue) evictionsMeter.mark() true } @@ -664,8 +685,8 @@ class FetchSessionCache(private val maxEntries: Int, */ def remove(session: FetchSession): Option[FetchSession] = synchronized { val evictableKey = session.synchronized { - lastUsed.remove(session.lastUsedKey()) - session.evictableKey() + lastUsed.remove(session.lastUsedKey) + session.evictableKey } evictableByAll.remove(evictableKey) evictableByPrivileged.remove(evictableKey) @@ -685,19 +706,19 @@ class FetchSessionCache(private val maxEntries: Int, def touch(session: FetchSession, now: Long): Unit = synchronized { session.synchronized { // Update the lastUsed map. - lastUsed.remove(session.lastUsedKey()) + lastUsed.remove(session.lastUsedKey) session.lastUsedMs = now - lastUsed.put(session.lastUsedKey(), session) + lastUsed.put(session.lastUsedKey, session) val oldSize = session.cachedSize if (oldSize != -1) { - val oldEvictableKey = session.evictableKey() + val oldEvictableKey = session.evictableKey evictableByPrivileged.remove(oldEvictableKey) evictableByAll.remove(oldEvictableKey) numPartitions = numPartitions - oldSize } - session.cachedSize = session.size() - val newEvictableKey = session.evictableKey() + session.cachedSize = session.size + val newEvictableKey = session.evictableKey if ((!session.privileged) || (now - session.creationMs > evictionMs)) { evictableByPrivileged.put(newEvictableKey, session) } @@ -717,35 +738,34 @@ class FetchManager(private val time: Time, isFollower: Boolean): FetchContext = { val context = if (reqMetadata.isFull) { var removedFetchSessionStr = "" - if (reqMetadata.sessionId() != INVALID_SESSION_ID) { + if (reqMetadata.sessionId != INVALID_SESSION_ID) { // Any session specified in a FULL fetch request will be closed. - if (cache.remove(reqMetadata.sessionId()).isDefined) { - removedFetchSessionStr = s" Removed fetch session ${reqMetadata.sessionId()}." + if (cache.remove(reqMetadata.sessionId).isDefined) { + removedFetchSessionStr = s" Removed fetch session ${reqMetadata.sessionId}." } } var suffix = "" - val context = if (reqMetadata.epoch() == FINAL_EPOCH) { + val context = if (reqMetadata.epoch == FINAL_EPOCH) { // If the epoch is FINAL_EPOCH, don't try to create a new session. suffix = " Will not try to create a new session." new SessionlessFetchContext(fetchData) } else { new FullFetchContext(time, cache, reqMetadata, fetchData, isFollower) } - debug(s"Created a new full FetchContext with ${partitionsToLogString(fetchData.keySet())}."+ + debug(s"Created a new full FetchContext with ${partitionsToLogString(fetchData.keySet)}."+ s"${removedFetchSessionStr}${suffix}") context } else { cache.synchronized { - cache.get(reqMetadata.sessionId()) match { + cache.get(reqMetadata.sessionId) match { case None => { - info(s"Created a new error FetchContext for session id ${reqMetadata.sessionId()}: " + - "no such session ID found.") + debug(s"Session error for ${reqMetadata.sessionId}: no such session ID found.") new SessionErrorContext(Errors.FETCH_SESSION_ID_NOT_FOUND, reqMetadata) } case Some(session) => session.synchronized { - if (session.epoch != reqMetadata.epoch()) { - debug(s"Created a new error FetchContext for session id ${session.id}: expected " + - s"epoch ${session.epoch}, but got epoch ${reqMetadata.epoch()}.") + if (session.epoch != reqMetadata.epoch) { + debug(s"Session error for ${reqMetadata.sessionId}: expected epoch " + + s"${session.epoch}, but got ${reqMetadata.epoch} instead."); new SessionErrorContext(Errors.INVALID_FETCH_SESSION_EPOCH, reqMetadata) } else { val (added, updated, removed) = session.update(fetchData, toForget, reqMetadata) @@ -756,7 +776,7 @@ class FetchManager(private val time: Time, cache.remove(session) new SessionlessFetchContext(fetchData) } else { - if (session.size() != session.cachedSize) { + if (session.size != session.cachedSize) { // If the number of partitions in the session changed, update the session's // position in the cache. cache.touch(session, session.lastUsedMs) diff --git a/core/src/main/scala/kafka/server/KafkaApis.scala b/core/src/main/scala/kafka/server/KafkaApis.scala index f4b8689f222f9..e94b0c3db7ffe 100644 --- a/core/src/main/scala/kafka/server/KafkaApis.scala +++ b/core/src/main/scala/kafka/server/KafkaApis.scala @@ -39,10 +39,10 @@ import kafka.server.QuotaFactory.{QuotaManagers, UnboundedQuota} import kafka.utils.{CoreUtils, Logging} import kafka.zk.{AdminZkClient, KafkaZkClient} import org.apache.kafka.common.acl.{AccessControlEntry, AclBinding} +import org.apache.kafka.common.config.ConfigResource import org.apache.kafka.common.errors._ import org.apache.kafka.common.internals.FatalExitError import org.apache.kafka.common.internals.Topic.{GROUP_METADATA_TOPIC_NAME, TRANSACTION_STATE_TOPIC_NAME, isInternal} -import org.apache.kafka.common.requests.FetchMetadata.INVALID_SESSION_ID import org.apache.kafka.common.metrics.Metrics import org.apache.kafka.common.network.{ListenerName, Send} import org.apache.kafka.common.protocol.{ApiKeys, Errors} @@ -51,8 +51,8 @@ import org.apache.kafka.common.requests.CreateAclsResponse.AclCreationResponse import org.apache.kafka.common.requests.DeleteAclsResponse.{AclDeletionResult, AclFilterResponse} import org.apache.kafka.common.requests.DescribeLogDirsResponse.LogDirInfo import org.apache.kafka.common.requests.ProduceResponse.PartitionResponse -import org.apache.kafka.common.requests.{Resource => RResource, ResourceType => RResourceType, _} -import org.apache.kafka.common.resource.{Resource => AdminResource} +import org.apache.kafka.common.requests._ +import org.apache.kafka.common.resource.PatternType.LITERAL import org.apache.kafka.common.security.auth.{KafkaPrincipal, SecurityProtocol} import org.apache.kafka.common.security.token.delegation.{DelegationToken, TokenInformation} import org.apache.kafka.common.utils.{Time, Utils} @@ -63,6 +63,7 @@ import scala.collection._ import scala.collection.mutable.ArrayBuffer import scala.util.{Failure, Success, Try} import org.apache.kafka.common.requests.CreateTopicsRequest.TopicDetails +import org.apache.kafka.common.resource.ResourcePattern /** * Logic to handle the various Kafka requests @@ -273,7 +274,7 @@ class KafkaApis(val requestChannel: RequestChannel, val offsetCommitRequest = request.body[OffsetCommitRequest] // reject the request if not authorized to the group - if (!authorize(request.session, Read, new Resource(Group, offsetCommitRequest.groupId, Literal))) { + if (!authorize(request.session, Read, Resource(Group, offsetCommitRequest.groupId, LITERAL))) { val error = Errors.GROUP_AUTHORIZATION_FAILED val results = offsetCommitRequest.offsetData.keySet.asScala.map { topicPartition => (topicPartition, error) @@ -286,7 +287,7 @@ class KafkaApis(val requestChannel: RequestChannel, val authorizedTopicRequestInfoBldr = immutable.Map.newBuilder[TopicPartition, OffsetCommitRequest.PartitionData] for ((topicPartition, partitionData) <- offsetCommitRequest.offsetData.asScala) { - if (!authorize(request.session, Read, new Resource(Topic, topicPartition.topic, Literal))) + if (!authorize(request.session, Read, Resource(Topic, topicPartition.topic, LITERAL))) unauthorizedTopicErrors += (topicPartition -> Errors.TOPIC_AUTHORIZATION_FAILED) else if (!metadataCache.contains(topicPartition)) nonExistingTopicErrors += (topicPartition -> Errors.UNKNOWN_TOPIC_OR_PARTITION) @@ -383,14 +384,16 @@ class KafkaApis(val requestChannel: RequestChannel, val produceRequest = request.body[ProduceRequest] val numBytesAppended = request.header.toStruct.sizeOf + request.sizeOfBodyInBytes - if (produceRequest.isTransactional) { - if (!authorize(request.session, Write, new Resource(TransactionalId, produceRequest.transactionalId, Literal))) { + if (produceRequest.hasTransactionalRecords) { + val isAuthorizedTransactional = produceRequest.transactionalId != null && + authorize(request.session, Write, Resource(TransactionalId, produceRequest.transactionalId, LITERAL)) + if (!isAuthorizedTransactional) { sendErrorResponseMaybeThrottle(request, Errors.TRANSACTIONAL_ID_AUTHORIZATION_FAILED.exception) return } // Note that authorization to a transactionalId implies ProducerId authorization - } else if (produceRequest.isIdempotent && !authorize(request.session, IdempotentWrite, Resource.ClusterResource)) { + } else if (produceRequest.hasIdempotentRecords && !authorize(request.session, IdempotentWrite, Resource.ClusterResource)) { sendErrorResponseMaybeThrottle(request, Errors.CLUSTER_AUTHORIZATION_FAILED.exception) return } @@ -400,7 +403,7 @@ class KafkaApis(val requestChannel: RequestChannel, val authorizedRequestInfo = mutable.Map[TopicPartition, MemoryRecords]() for ((topicPartition, memoryRecords) <- produceRequest.partitionRecordsOrFail.asScala) { - if (!authorize(request.session, Write, new Resource(Topic, topicPartition.topic, Literal))) + if (!authorize(request.session, Write, Resource(Topic, topicPartition.topic, LITERAL))) unauthorizedTopicResponses += topicPartition -> new PartitionResponse(Errors.TOPIC_AUTHORIZATION_FAILED) else if (!metadataCache.contains(topicPartition)) nonExistingTopicResponses += topicPartition -> new PartitionResponse(Errors.UNKNOWN_TOPIC_OR_PARTITION) @@ -505,44 +508,41 @@ class KafkaApis(val requestChannel: RequestChannel, fetchRequest.toForget(), fetchRequest.isFromFollower()) + def errorResponse[T >: MemoryRecords <: BaseRecords](error: Errors): FetchResponse.PartitionData[T] = { + new FetchResponse.PartitionData[T](error, FetchResponse.INVALID_HIGHWATERMARK, FetchResponse.INVALID_LAST_STABLE_OFFSET, + FetchResponse.INVALID_LOG_START_OFFSET, null, MemoryRecords.EMPTY) + } + val erroneous = mutable.ArrayBuffer[(TopicPartition, FetchResponse.PartitionData[Records])]() val interesting = mutable.ArrayBuffer[(TopicPartition, FetchRequest.PartitionData)]() if (fetchRequest.isFromFollower()) { // The follower must have ClusterAction on ClusterResource in order to fetch partition data. if (authorize(request.session, ClusterAction, Resource.ClusterResource)) { - fetchContext.foreachPartition((topicPartition, data) => { - if (!metadataCache.contains(topicPartition)) { - erroneous += topicPartition -> new FetchResponse.PartitionData(Errors.UNKNOWN_TOPIC_OR_PARTITION, - FetchResponse.INVALID_HIGHWATERMARK, FetchResponse.INVALID_LAST_STABLE_OFFSET, - FetchResponse.INVALID_LOG_START_OFFSET, null, MemoryRecords.EMPTY) - } else { + fetchContext.foreachPartition { (topicPartition, data) => + if (!metadataCache.contains(topicPartition)) + erroneous += topicPartition -> errorResponse(Errors.UNKNOWN_TOPIC_OR_PARTITION) + else interesting += (topicPartition -> data) - } - }) + } } else { - fetchContext.foreachPartition((part, _) => { - erroneous += part -> new FetchResponse.PartitionData(Errors.TOPIC_AUTHORIZATION_FAILED, - FetchResponse.INVALID_HIGHWATERMARK, FetchResponse.INVALID_LAST_STABLE_OFFSET, - FetchResponse.INVALID_LOG_START_OFFSET, null, MemoryRecords.EMPTY) - }) + fetchContext.foreachPartition { (part, _) => + erroneous += part -> errorResponse(Errors.TOPIC_AUTHORIZATION_FAILED) + } } } else { // Regular Kafka consumers need READ permission on each partition they are fetching. - fetchContext.foreachPartition((topicPartition, data) => { - if (!authorize(request.session, Read, new Resource(Topic, topicPartition.topic, Literal))) - erroneous += topicPartition -> new FetchResponse.PartitionData(Errors.TOPIC_AUTHORIZATION_FAILED, - FetchResponse.INVALID_HIGHWATERMARK, FetchResponse.INVALID_LAST_STABLE_OFFSET, - FetchResponse.INVALID_LOG_START_OFFSET, null, MemoryRecords.EMPTY) + fetchContext.foreachPartition { (topicPartition, data) => + if (!authorize(request.session, Read, Resource(Topic, topicPartition.topic, LITERAL))) + erroneous += topicPartition -> errorResponse(Errors.TOPIC_AUTHORIZATION_FAILED) else if (!metadataCache.contains(topicPartition)) - erroneous += topicPartition -> new FetchResponse.PartitionData(Errors.UNKNOWN_TOPIC_OR_PARTITION, - FetchResponse.INVALID_HIGHWATERMARK, FetchResponse.INVALID_LAST_STABLE_OFFSET, - FetchResponse.INVALID_LOG_START_OFFSET, null, MemoryRecords.EMPTY) + erroneous += topicPartition -> errorResponse(Errors.UNKNOWN_TOPIC_OR_PARTITION) else interesting += (topicPartition -> data) - }) + } } - def convertRecords(tp: TopicPartition, unconvertedRecords: Records): BaseRecords = { + def maybeConvertFetchedData(tp: TopicPartition, + partitionData: FetchResponse.PartitionData[Records]): FetchResponse.PartitionData[BaseRecords] = { // Down-conversion of the fetched records is needed when the stored magic version is // greater than that supported by the client (as indicated by the fetch request version). If the // configured magic version for the topic is less than or equal to that supported by the version of the @@ -550,8 +550,10 @@ class KafkaApis(val requestChannel: RequestChannel, // know it must be supported. However, if the magic version is changed from a higher version back to a // lower version, this check will no longer be valid and we will fail to down-convert the messages // which were written in the new format prior to the version downgrade. - replicaManager.getMagic(tp).flatMap { magic => - val downConvertMagic = { + val unconvertedRecords = partitionData.records + val logConfig = replicaManager.getLogConfig(tp) + val downConvertMagic = + logConfig.map(_.messageFormatVersion.recordVersion.value).flatMap { magic => if (magic > RecordBatch.MAGIC_VALUE_V0 && versionId <= 1 && !unconvertedRecords.hasCompatibleMagic(RecordBatch.MAGIC_VALUE_V0)) Some(RecordBatch.MAGIC_VALUE_V0) else if (magic > RecordBatch.MAGIC_VALUE_V1 && versionId <= 3 && !unconvertedRecords.hasCompatibleMagic(RecordBatch.MAGIC_VALUE_V1)) @@ -560,28 +562,36 @@ class KafkaApis(val requestChannel: RequestChannel, None } - downConvertMagic.map { magic => - trace(s"Down converting records from partition $tp to message format version $magic for fetch request from $clientId") - - // Because down-conversion is extremely memory intensive, we want to try and delay the down-conversion as much - // as possible. With KIP-283, we have the ability to lazily down-convert in a chunked manner. The lazy, chunked - // down-conversion always guarantees that at least one batch of messages is down-converted and sent out to the - // client. - new LazyDownConversionRecords(tp, unconvertedRecords, magic, fetchContext.getFetchOffset(tp).get, time) - } - }.getOrElse(unconvertedRecords) + // For fetch requests from clients, check if down-conversion is disabled for the particular partition + if (downConvertMagic.isDefined && !fetchRequest.isFromFollower && !logConfig.forall(_.messageDownConversionEnable)) { + trace(s"Conversion to message format ${downConvertMagic.get} is disabled for partition $tp. Sending unsupported version response to $clientId.") + errorResponse(Errors.UNSUPPORTED_VERSION) + } else { + val convertedRecords = + downConvertMagic.map { magic => + trace(s"Down converting records from partition $tp to message format version $magic for fetch request from $clientId") + // Because down-conversion is extremely memory intensive, we want to try and delay the down-conversion as much + // as possible. With KIP-283, we have the ability to lazily down-convert in a chunked manner. The lazy, chunked + // down-conversion always guarantees that at least one batch of messages is down-converted and sent out to the + // client. + new LazyDownConversionRecords(tp, unconvertedRecords, magic, fetchContext.getFetchOffset(tp).get, time) + }.getOrElse(unconvertedRecords) + new FetchResponse.PartitionData[BaseRecords](partitionData.error, partitionData.highWatermark, + partitionData.lastStableOffset, partitionData.logStartOffset, partitionData.abortedTransactions, + convertedRecords) + } } // the callback for process a fetch response, invoked before throttling def processResponseCallback(responsePartitionData: Seq[(TopicPartition, FetchPartitionData)]): Unit = { val partitions = new util.LinkedHashMap[TopicPartition, FetchResponse.PartitionData[Records]] - responsePartitionData.foreach{ case (tp, data) => + responsePartitionData.foreach { case (tp, data) => val abortedTransactions = data.abortedTransactions.map(_.asJava).orNull val lastStableOffset = data.lastStableOffset.getOrElse(FetchResponse.INVALID_LAST_STABLE_OFFSET) partitions.put(tp, new FetchResponse.PartitionData(data.error, data.highWatermark, lastStableOffset, data.logStartOffset, abortedTransactions, data.records)) } - erroneous.foreach{case (tp, data) => partitions.put(tp, data)} + erroneous.foreach { case (tp, data) => partitions.put(tp, data) } // When this callback is triggered, the remote API call has completed. // Record time before any byte-rate throttling. @@ -596,14 +606,10 @@ class KafkaApis(val requestChannel: RequestChannel, if (unconvertedPartitionData.error != Errors.NONE) debug(s"Fetch request with correlation id ${request.header.correlationId} from client $clientId " + s"on partition $tp failed due to ${unconvertedPartitionData.error.exceptionName}") - val convertedRecords = convertRecords(tp, unconvertedPartitionData.records) - val convertedPartitionData = new FetchResponse.PartitionData[BaseRecords](unconvertedPartitionData.error, - unconvertedPartitionData.highWatermark, FetchResponse.INVALID_LAST_STABLE_OFFSET, unconvertedPartitionData.logStartOffset, - unconvertedPartitionData.abortedTransactions, convertedRecords) - convertedData.put(tp, convertedPartitionData) + convertedData.put(tp, maybeConvertFetchedData(tp, unconvertedPartitionData)) } - // Prepare fetch resopnse from converted data + // Prepare fetch response from converted data val response = new FetchResponse(unconvertedFetchResponse.error(), convertedData, throttleTimeMs, unconvertedFetchResponse.sessionId()) response.responseData.asScala.foreach { case (topicPartition, data) => @@ -655,8 +661,7 @@ class KafkaApis(val requestChannel: RequestChannel, quotas.request.throttle(request, requestThrottleTimeMs, sendResponse) } // If throttling is required, return an empty response. - unconvertedFetchResponse = new FetchResponse(Errors.NONE, new util.LinkedHashMap[TopicPartition, - FetchResponse.PartitionData[Records]](), maxThrottleTimeMs, INVALID_SESSION_ID) + unconvertedFetchResponse = fetchContext.getThrottledResponse(maxThrottleTimeMs) } else { // Get the actual response. This will update the fetch context. unconvertedFetchResponse = fetchContext.updateAndGenerateResponseData(partitions) @@ -741,7 +746,7 @@ class KafkaApis(val requestChannel: RequestChannel, val offsetRequest = request.body[ListOffsetRequest] val (authorizedRequestInfo, unauthorizedRequestInfo) = offsetRequest.offsetData.asScala.partition { - case (topicPartition, _) => authorize(request.session, Describe, new Resource(Topic, topicPartition.topic, Literal)) + case (topicPartition, _) => authorize(request.session, Describe, Resource(Topic, topicPartition.topic, LITERAL)) } val unauthorizedResponseStatus = unauthorizedRequestInfo.mapValues(_ => @@ -794,7 +799,7 @@ class KafkaApis(val requestChannel: RequestChannel, val offsetRequest = request.body[ListOffsetRequest] val (authorizedRequestInfo, unauthorizedRequestInfo) = offsetRequest.partitionTimestamps.asScala.partition { - case (topicPartition, _) => authorize(request.session, Describe, new Resource(Topic, topicPartition.topic, Literal)) + case (topicPartition, _) => authorize(request.session, Describe, Resource(Topic, topicPartition.topic, LITERAL)) } val unauthorizedResponseStatus = unauthorizedRequestInfo.mapValues(_ => { @@ -988,8 +993,10 @@ class KafkaApis(val requestChannel: RequestChannel, } private def getTopicMetadata(allowAutoTopicCreation: Boolean, topics: Set[String], listenerName: ListenerName, - errorUnavailableEndpoints: Boolean): Seq[MetadataResponse.TopicMetadata] = { - val topicResponses = metadataCache.getTopicMetadata(topics, listenerName, errorUnavailableEndpoints) + errorUnavailableEndpoints: Boolean, + errorUnavailableListeners: Boolean): Seq[MetadataResponse.TopicMetadata] = { + val topicResponses = metadataCache.getTopicMetadata(topics, listenerName, + errorUnavailableEndpoints, errorUnavailableListeners) if (topics.isEmpty || topicResponses.size == topics.size) { topicResponses } else { @@ -1033,7 +1040,7 @@ class KafkaApis(val requestChannel: RequestChannel, } var (authorizedTopics, unauthorizedForDescribeTopics) = - topics.partition(topic => authorize(request.session, Describe, new Resource(Topic, topic, Literal))) + topics.partition(topic => authorize(request.session, Describe, Resource(Topic, topic, LITERAL))) var unauthorizedForCreateTopics = Set[String]() @@ -1065,12 +1072,15 @@ class KafkaApis(val requestChannel: RequestChannel, // In version 0, we returned an error when brokers with replicas were unavailable, // while in higher versions we simply don't include the broker in the returned broker list val errorUnavailableEndpoints = requestVersion == 0 + // In versions 5 and below, we returned LEADER_NOT_AVAILABLE if a matching listener was not found on the leader. + // From version 6 onwards, we return LISTENER_NOT_FOUND to enable diagnosis of configuration errors. + val errorUnavailableListeners = requestVersion >= 6 val topicMetadata = if (authorizedTopics.isEmpty) Seq.empty[MetadataResponse.TopicMetadata] else getTopicMetadata(metadataRequest.allowAutoTopicCreation, authorizedTopics, request.context.listenerName, - errorUnavailableEndpoints) + errorUnavailableEndpoints, errorUnavailableListeners) val completeTopicMetadata = topicMetadata ++ unauthorizedForCreateTopicMetadata ++ unauthorizedForDescribeTopicMetadata @@ -1097,12 +1107,12 @@ class KafkaApis(val requestChannel: RequestChannel, val offsetFetchRequest = request.body[OffsetFetchRequest] def authorizeTopicDescribe(partition: TopicPartition) = - authorize(request.session, Describe, new Resource(Topic, partition.topic, Literal)) + authorize(request.session, Describe, Resource(Topic, partition.topic, LITERAL)) def createResponse(requestThrottleMs: Int): AbstractResponse = { val offsetFetchResponse = // reject the request if not authorized to the group - if (!authorize(request.session, Describe, new Resource(Group, offsetFetchRequest.groupId, Literal))) + if (!authorize(request.session, Describe, Resource(Group, offsetFetchRequest.groupId, LITERAL))) offsetFetchRequest.getErrorResponse(requestThrottleMs, Errors.GROUP_AUTHORIZATION_FAILED) else { if (header.apiVersion == 0) { @@ -1170,10 +1180,10 @@ class KafkaApis(val requestChannel: RequestChannel, val findCoordinatorRequest = request.body[FindCoordinatorRequest] if (findCoordinatorRequest.coordinatorType == FindCoordinatorRequest.CoordinatorType.GROUP && - !authorize(request.session, Describe, new Resource(Group, findCoordinatorRequest.coordinatorKey, Literal))) + !authorize(request.session, Describe, Resource(Group, findCoordinatorRequest.coordinatorKey, LITERAL))) sendErrorResponseMaybeThrottle(request, Errors.GROUP_AUTHORIZATION_FAILED.exception) else if (findCoordinatorRequest.coordinatorType == FindCoordinatorRequest.CoordinatorType.TRANSACTION && - !authorize(request.session, Describe, new Resource(TransactionalId, findCoordinatorRequest.coordinatorKey, Literal))) + !authorize(request.session, Describe, Resource(TransactionalId, findCoordinatorRequest.coordinatorKey, LITERAL))) sendErrorResponseMaybeThrottle(request, Errors.TRANSACTIONAL_ID_AUTHORIZATION_FAILED.exception) else { // get metadata (and create the topic if necessary) @@ -1220,7 +1230,7 @@ class KafkaApis(val requestChannel: RequestChannel, val describeRequest = request.body[DescribeGroupsRequest] val groups = describeRequest.groupIds.asScala.map { groupId => - if (!authorize(request.session, Describe, new Resource(Group, groupId, Literal))) { + if (!authorize(request.session, Describe, Resource(Group, groupId, LITERAL))) { groupId -> DescribeGroupsResponse.GroupMetadata.forError(Errors.GROUP_AUTHORIZATION_FAILED) } else { val (error, summary) = groupCoordinator.handleDescribeGroup(groupId) @@ -1266,7 +1276,7 @@ class KafkaApis(val requestChannel: RequestChannel, sendResponseMaybeThrottle(request, createResponse) } - if (!authorize(request.session, Read, new Resource(Group, joinGroupRequest.groupId(), Literal))) { + if (!authorize(request.session, Read, Resource(Group, joinGroupRequest.groupId(), LITERAL))) { sendResponseMaybeThrottle(request, requestThrottleMs => new JoinGroupResponse( requestThrottleMs, @@ -1302,7 +1312,7 @@ class KafkaApis(val requestChannel: RequestChannel, new SyncGroupResponse(requestThrottleMs, error, ByteBuffer.wrap(memberState))) } - if (!authorize(request.session, Read, new Resource(Group, syncGroupRequest.groupId(), Literal))) { + if (!authorize(request.session, Read, Resource(Group, syncGroupRequest.groupId(), LITERAL))) { sendResponseCallback(Array[Byte](), Errors.GROUP_AUTHORIZATION_FAILED) } else { groupCoordinator.handleSyncGroup( @@ -1317,10 +1327,10 @@ class KafkaApis(val requestChannel: RequestChannel, def handleDeleteGroupsRequest(request: RequestChannel.Request): Unit = { val deleteGroupsRequest = request.body[DeleteGroupsRequest] - var groups = deleteGroupsRequest.groups.asScala.toSet + val groups = deleteGroupsRequest.groups.asScala.toSet val (authorizedGroups, unauthorizedGroups) = groups.partition { group => - authorize(request.session, Delete, new Resource(Group, group, Literal)) + authorize(request.session, Delete, Resource(Group, group, LITERAL)) } val groupDeletionResult = groupCoordinator.handleDeleteGroups(authorizedGroups) ++ @@ -1344,7 +1354,7 @@ class KafkaApis(val requestChannel: RequestChannel, sendResponseMaybeThrottle(request, createResponse) } - if (!authorize(request.session, Read, new Resource(Group, heartbeatRequest.groupId, Literal))) { + if (!authorize(request.session, Read, Resource(Group, heartbeatRequest.groupId, LITERAL))) { sendResponseMaybeThrottle(request, requestThrottleMs => new HeartbeatResponse(requestThrottleMs, Errors.GROUP_AUTHORIZATION_FAILED)) } else { @@ -1371,7 +1381,7 @@ class KafkaApis(val requestChannel: RequestChannel, sendResponseMaybeThrottle(request, createResponse) } - if (!authorize(request.session, Read, new Resource(Group, leaveGroupRequest.groupId, Literal))) { + if (!authorize(request.session, Read, Resource(Group, leaveGroupRequest.groupId, LITERAL))) { sendResponseMaybeThrottle(request, requestThrottleMs => new LeaveGroupResponse(requestThrottleMs, Errors.GROUP_AUTHORIZATION_FAILED)) } else { @@ -1454,7 +1464,7 @@ class KafkaApis(val requestChannel: RequestChannel, duplicateTopics.keySet.map((_, new ApiError(Errors.INVALID_REQUEST, errorMessage))).toMap } else Map.empty - val unauthorizedTopicsResults = unauthorizedTopics.keySet.map(_ -> new ApiError(Errors.TOPIC_AUTHORIZATION_FAILED, null)) + val unauthorizedTopicsResults = unauthorizedTopics.keySet.map(_ -> new ApiError(Errors.TOPIC_AUTHORIZATION_FAILED, null)) val completeResults = results ++ duplicatedTopicsResults ++ unauthorizedTopicsResults sendResponseCallback(completeResults) } @@ -1491,7 +1501,7 @@ class KafkaApis(val requestChannel: RequestChannel, val dupes = createPartitionsRequest.duplicates.asScala val notDuped = createPartitionsRequest.newPartitions.asScala -- dupes val (authorized, unauthorized) = notDuped.partition { case (topic, _) => - authorize(request.session, Alter, new Resource(Topic, topic, Literal)) + authorize(request.session, Alter, Resource(Topic, topic, LITERAL)) } val (queuedForDeletion, valid) = authorized.partition { case (topic, _) => @@ -1515,7 +1525,7 @@ class KafkaApis(val requestChannel: RequestChannel, val authorizedForDeleteTopics = mutable.Set[String]() for (topic <- deleteTopicRequest.topics.asScala) { - if (!authorize(request.session, Delete, new Resource(Topic, topic, Literal))) + if (!authorize(request.session, Delete, Resource(Topic, topic, LITERAL))) unauthorizedTopicErrors += topic -> Errors.TOPIC_AUTHORIZATION_FAILED else if (!metadataCache.contains(topic)) nonExistingTopicErrors += topic -> Errors.UNKNOWN_TOPIC_OR_PARTITION @@ -1560,7 +1570,7 @@ class KafkaApis(val requestChannel: RequestChannel, val authorizedForDeleteTopicOffsets = mutable.Map[TopicPartition, Long]() for ((topicPartition, offset) <- deleteRecordsRequest.partitionOffsets.asScala) { - if (!authorize(request.session, Delete, new Resource(Topic, topicPartition.topic, Literal))) + if (!authorize(request.session, Delete, Resource(Topic, topicPartition.topic, LITERAL))) unauthorizedTopicResponses += topicPartition -> new DeleteRecordsResponse.PartitionResponse( DeleteRecordsResponse.INVALID_LOW_WATERMARK, Errors.TOPIC_AUTHORIZATION_FAILED) else if (!metadataCache.contains(topicPartition)) @@ -1603,7 +1613,7 @@ class KafkaApis(val requestChannel: RequestChannel, val transactionalId = initProducerIdRequest.transactionalId if (transactionalId != null) { - if (!authorize(request.session, Write, new Resource(TransactionalId, transactionalId, Literal))) { + if (!authorize(request.session, Write, Resource(TransactionalId, transactionalId, LITERAL))) { sendErrorResponseMaybeThrottle(request, Errors.TRANSACTIONAL_ID_AUTHORIZATION_FAILED.exception) return } @@ -1628,7 +1638,7 @@ class KafkaApis(val requestChannel: RequestChannel, val endTxnRequest = request.body[EndTxnRequest] val transactionalId = endTxnRequest.transactionalId - if (authorize(request.session, Write, new Resource(TransactionalId, transactionalId, Literal))) { + if (authorize(request.session, Write, Resource(TransactionalId, transactionalId, LITERAL))) { def sendResponseCallback(error: Errors) { def createResponse(requestThrottleMs: Int): AbstractResponse = { val responseBody = new EndTxnResponse(requestThrottleMs, error) @@ -1763,7 +1773,7 @@ class KafkaApis(val requestChannel: RequestChannel, val addPartitionsToTxnRequest = request.body[AddPartitionsToTxnRequest] val transactionalId = addPartitionsToTxnRequest.transactionalId val partitionsToAdd = addPartitionsToTxnRequest.partitions.asScala - if (!authorize(request.session, Write, new Resource(TransactionalId, transactionalId, Literal))) + if (!authorize(request.session, Write, Resource(TransactionalId, transactionalId, LITERAL))) sendResponseMaybeThrottle(request, requestThrottleMs => addPartitionsToTxnRequest.getErrorResponse(requestThrottleMs, Errors.TRANSACTIONAL_ID_AUTHORIZATION_FAILED.exception)) else { @@ -1773,7 +1783,7 @@ class KafkaApis(val requestChannel: RequestChannel, for (topicPartition <- partitionsToAdd) { if (org.apache.kafka.common.internals.Topic.isInternal(topicPartition.topic) || - !authorize(request.session, Write, new Resource(Topic, topicPartition.topic, Literal))) + !authorize(request.session, Write, Resource(Topic, topicPartition.topic, LITERAL))) unauthorizedTopicErrors += topicPartition -> Errors.TOPIC_AUTHORIZATION_FAILED else if (!metadataCache.contains(topicPartition)) nonExistingTopicErrors += topicPartition -> Errors.UNKNOWN_TOPIC_OR_PARTITION @@ -1817,10 +1827,10 @@ class KafkaApis(val requestChannel: RequestChannel, val groupId = addOffsetsToTxnRequest.consumerGroupId val offsetTopicPartition = new TopicPartition(GROUP_METADATA_TOPIC_NAME, groupCoordinator.partitionFor(groupId)) - if (!authorize(request.session, Write, new Resource(TransactionalId, transactionalId, Literal))) + if (!authorize(request.session, Write, Resource(TransactionalId, transactionalId, LITERAL))) sendResponseMaybeThrottle(request, requestThrottleMs => new AddOffsetsToTxnResponse(requestThrottleMs, Errors.TRANSACTIONAL_ID_AUTHORIZATION_FAILED)) - else if (!authorize(request.session, Read, new Resource(Group, groupId, Literal))) + else if (!authorize(request.session, Read, Resource(Group, groupId, LITERAL))) sendResponseMaybeThrottle(request, requestThrottleMs => new AddOffsetsToTxnResponse(requestThrottleMs, Errors.GROUP_AUTHORIZATION_FAILED)) else { @@ -1849,9 +1859,9 @@ class KafkaApis(val requestChannel: RequestChannel, // authorize for the transactionalId and the consumer group. Note that we skip producerId authorization // since it is implied by transactionalId authorization - if (!authorize(request.session, Write, new Resource(TransactionalId, txnOffsetCommitRequest.transactionalId, Literal))) + if (!authorize(request.session, Write, Resource(TransactionalId, txnOffsetCommitRequest.transactionalId, LITERAL))) sendErrorResponseMaybeThrottle(request, Errors.TRANSACTIONAL_ID_AUTHORIZATION_FAILED.exception) - else if (!authorize(request.session, Read, new Resource(Group, txnOffsetCommitRequest.consumerGroupId, Literal))) + else if (!authorize(request.session, Read, Resource(Group, txnOffsetCommitRequest.consumerGroupId, LITERAL))) sendErrorResponseMaybeThrottle(request, Errors.GROUP_AUTHORIZATION_FAILED.exception) else { val unauthorizedTopicErrors = mutable.Map[TopicPartition, Errors]() @@ -1859,7 +1869,7 @@ class KafkaApis(val requestChannel: RequestChannel, val authorizedTopicCommittedOffsets = mutable.Map[TopicPartition, TxnOffsetCommitRequest.CommittedOffset]() for ((topicPartition, commitedOffset) <- txnOffsetCommitRequest.offsets.asScala) { - if (!authorize(request.session, Read, new Resource(Topic, topicPartition.topic, Literal))) + if (!authorize(request.session, Read, Resource(Topic, topicPartition.topic, LITERAL))) unauthorizedTopicErrors += topicPartition -> Errors.TOPIC_AUTHORIZATION_FAILED else if (!metadataCache.contains(topicPartition)) nonExistingTopicErrors += topicPartition -> Errors.UNKNOWN_TOPIC_OR_PARTITION @@ -1920,10 +1930,9 @@ class KafkaApis(val requestChannel: RequestChannel, val filter = describeAclsRequest.filter() val returnedAcls = auth.getAcls.toSeq.flatMap { case (resource, acls) => acls.flatMap { acl => - val fixture = new AclBinding(new AdminResource(resource.resourceType.toJava, resource.name, resource.resourceNameType.toJava), + val fixture = new AclBinding(new ResourcePattern(resource.resourceType.toJava, resource.name, resource.patternType), new AccessControlEntry(acl.principal.toString, acl.host.toString, acl.operation.toJava, acl.permissionType.toJava)) - if (filter.matches(fixture)) Some(fixture) - else None + Some(fixture).filter(filter.matches) } } sendResponseMaybeThrottle(request, requestThrottleMs => @@ -1994,7 +2003,7 @@ class KafkaApis(val requestChannel: RequestChannel, val filtersWithIndex = filters.zipWithIndex for ((resource, acls) <- aclMap; acl <- acls) { val binding = new AclBinding( - new AdminResource(resource.resourceType.toJava, resource.name, resource.resourceNameType.toJava), + new ResourcePattern(resource.resourceType.toJava, resource.name, resource.patternType), new AccessControlEntry(acl.principal.toString, acl.host.toString, acl.operation.toJava, acl.permissionType.toJava)) @@ -2039,10 +2048,10 @@ class KafkaApis(val requestChannel: RequestChannel, val alterConfigsRequest = request.body[AlterConfigsRequest] val (authorizedResources, unauthorizedResources) = alterConfigsRequest.configs.asScala.partition { case (resource, _) => resource.`type` match { - case RResourceType.BROKER => + case ConfigResource.Type.BROKER => authorize(request.session, AlterConfigs, Resource.ClusterResource) - case RResourceType.TOPIC => - authorize(request.session, AlterConfigs, new Resource(Topic, resource.name, Literal)) + case ConfigResource.Type.TOPIC => + authorize(request.session, AlterConfigs, Resource(Topic, resource.name, LITERAL)) case rt => throw new InvalidRequestException(s"Unexpected resource type $rt") } } @@ -2054,10 +2063,10 @@ class KafkaApis(val requestChannel: RequestChannel, new AlterConfigsResponse(requestThrottleMs, (authorizedResult ++ unauthorizedResult).asJava)) } - private def configsAuthorizationApiError(session: RequestChannel.Session, resource: RResource): ApiError = { + private def configsAuthorizationApiError(session: RequestChannel.Session, resource: ConfigResource): ApiError = { val error = resource.`type` match { - case RResourceType.BROKER => Errors.CLUSTER_AUTHORIZATION_FAILED - case RResourceType.TOPIC => Errors.TOPIC_AUTHORIZATION_FAILED + case ConfigResource.Type.BROKER => Errors.CLUSTER_AUTHORIZATION_FAILED + case ConfigResource.Type.TOPIC => Errors.TOPIC_AUTHORIZATION_FAILED case rt => throw new InvalidRequestException(s"Unexpected resource type $rt for resource ${resource.name}") } new ApiError(error, null) @@ -2067,9 +2076,9 @@ class KafkaApis(val requestChannel: RequestChannel, val describeConfigsRequest = request.body[DescribeConfigsRequest] val (authorizedResources, unauthorizedResources) = describeConfigsRequest.resources.asScala.partition { resource => resource.`type` match { - case RResourceType.BROKER => authorize(request.session, DescribeConfigs, Resource.ClusterResource) - case RResourceType.TOPIC => - authorize(request.session, DescribeConfigs, new Resource(Topic, resource.name, Literal)) + case ConfigResource.Type.BROKER => authorize(request.session, DescribeConfigs, Resource.ClusterResource) + case ConfigResource.Type.TOPIC => + authorize(request.session, DescribeConfigs, Resource(Topic, resource.name, LITERAL)) case rt => throw new InvalidRequestException(s"Unexpected resource type $rt for resource ${resource.name}") } } @@ -2216,7 +2225,7 @@ class KafkaApis(val requestChannel: RequestChannel, } else { val owners = if (describeTokenRequest.owners == null) None else Some(describeTokenRequest.owners.asScala.toList) - def authorizeToken(tokenId: String) = authorize(request.session, Describe, new Resource(kafka.security.auth.DelegationToken, tokenId, Literal)) + def authorizeToken(tokenId: String) = authorize(request.session, Describe, Resource(kafka.security.auth.DelegationToken, tokenId, LITERAL)) def eligible(token: TokenInformation) = DelegationTokenManager.filterToken(requestPrincipal, owners, token, authorizeToken) val tokens = tokenManager.getTokens(eligible) sendResponseCallback(Errors.NONE, tokens) diff --git a/core/src/main/scala/kafka/server/KafkaConfig.scala b/core/src/main/scala/kafka/server/KafkaConfig.scala index 19bb80749584d..744442fd8e990 100755 --- a/core/src/main/scala/kafka/server/KafkaConfig.scala +++ b/core/src/main/scala/kafka/server/KafkaConfig.scala @@ -22,10 +22,9 @@ import java.util.{Collections, Properties} import kafka.api.{ApiVersion, KAFKA_0_10_0_IV1} import kafka.cluster.EndPoint -import kafka.consumer.ConsumerConfig import kafka.coordinator.group.OffsetConfig import kafka.coordinator.transaction.{TransactionLog, TransactionStateManager} -import kafka.message.{BrokerCompressionCodec, CompressionCodec, Message, MessageSet} +import kafka.message.{BrokerCompressionCodec, CompressionCodec} import kafka.utils.CoreUtils import kafka.utils.Implicits._ import org.apache.kafka.clients.CommonClientConfigs @@ -35,7 +34,7 @@ import org.apache.kafka.common.config.internals.BrokerSecurityConfigs import org.apache.kafka.common.config.{AbstractConfig, ConfigDef, ConfigException, SaslConfigs, SslConfigs, TopicConfig} import org.apache.kafka.common.metrics.Sensor import org.apache.kafka.common.network.ListenerName -import org.apache.kafka.common.record.TimestampType +import org.apache.kafka.common.record.{LegacyRecord, Records, TimestampType} import org.apache.kafka.common.security.auth.SecurityProtocol import scala.collection.JavaConverters._ @@ -52,7 +51,7 @@ object Defaults { val BrokerIdGenerationEnable = true val MaxReservedBrokerId = 1000 val BrokerId = -1 - val MessageMaxBytes = 1000000 + MessageSet.LogOverhead + val MessageMaxBytes = 1000000 + Records.LOG_OVERHEAD val NumNetworkThreads = 3 val NumIoThreads = 8 val BackgroundThreads = 10 @@ -116,15 +115,16 @@ object Defaults { val NumRecoveryThreadsPerDataDir = 1 val AutoCreateTopicsEnable = true val MinInSyncReplicas = 1 + val MessageDownConversionEnable = true /** ********* 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 ReplicaSocketTimeoutMs = 30 * 1000 + val ReplicaSocketReceiveBufferBytes = 64 * 1024 + val ReplicaFetchMaxBytes = 1024 * 1024 val ReplicaFetchWaitMaxMs = 500 val ReplicaFetchMinBytes = 1 val ReplicaFetchResponseMaxBytes = 10 * 1024 * 1024 @@ -330,6 +330,7 @@ object KafkaConfig { val MinInSyncReplicasProp = "min.insync.replicas" val CreateTopicPolicyClassNameProp = "create.topic.policy.class.name" val AlterConfigPolicyClassNameProp = "alter.config.policy.class.name" + val LogMessageDownConversionEnableProp = LogConfigPrefix + "message.downconversion.enable" /** ********* Replication configuration ***********/ val ControllerSocketTimeoutMsProp = "controller.socket.timeout.ms" val DefaultReplicationFactorProp = "default.replication.factor" @@ -599,6 +600,7 @@ object KafkaConfig { "implement the org.apache.kafka.server.policy.CreateTopicPolicy interface." val AlterConfigPolicyClassNameDoc = "The alter configs policy class that should be used for validation. The class should " + "implement the org.apache.kafka.server.policy.AlterConfigPolicy interface." + val LogMessageDownConversionEnableDoc = TopicConfig.MESSAGE_DOWNCONVERSION_ENABLE_DOC; /** ********* Replication configuration ***********/ val ControllerSocketTimeoutMsDoc = "The socket timeout for controller-to-broker channels" @@ -649,7 +651,7 @@ object KafkaConfig { val GroupInitialRebalanceDelayMsDoc = "The amount of time the group coordinator will wait for more consumers to join a new group before performing the first rebalance. A longer delay means potentially fewer rebalances, but increases the time until processing begins." /** ********* 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 OffsetsLoadBufferSizeDoc = "Batch size for reading from the offsets segments when loading offsets into the cache (soft-limit, overridden if records are too large)." val OffsetsTopicReplicationFactorDoc = "The replication factor for the offsets topic (set higher to ensure availability). " + "Internal topic creation will fail until the cluster size meets this replication factor requirement." val OffsetsTopicPartitionsDoc = "The number of partitions for the offset commit topic (should not change after deployment)" @@ -665,7 +667,7 @@ object KafkaConfig { val TransactionsMaxTimeoutMsDoc = "The maximum allowed timeout for transactions. " + "If a client’s requested transaction time exceed this, then the broker will return an error in InitProducerIdRequest. This prevents a client from too large of a timeout, which can stall consumers reading from topics included in the transaction." val TransactionsTopicMinISRDoc = "Overridden " + MinInSyncReplicasProp + " config for the transaction topic." - val TransactionsLoadBufferSizeDoc = "Batch size for reading from the transaction log segments when loading producer ids and transactions into the cache." + val TransactionsLoadBufferSizeDoc = "Batch size for reading from the transaction log segments when loading producer ids and transactions into the cache (soft-limit, overridden if records are too large)." val TransactionsTopicReplicationFactorDoc = "The replication factor for the transaction topic (set higher to ensure availability). " + "Internal topic creation will fail until the cluster size meets this replication factor requirement." val TransactionsTopicPartitionsDoc = "The number of partitions for the transaction topic (should not change after deployment)." @@ -820,7 +822,7 @@ object KafkaConfig { .define(NumPartitionsProp, INT, Defaults.NumPartitions, atLeast(1), MEDIUM, NumPartitionsDoc) .define(LogDirProp, STRING, Defaults.LogDir, HIGH, LogDirDoc) .define(LogDirsProp, STRING, null, HIGH, LogDirsDoc) - .define(LogSegmentBytesProp, INT, Defaults.LogSegmentBytes, atLeast(Message.MinMessageOverhead), HIGH, LogSegmentBytesDoc) + .define(LogSegmentBytesProp, INT, Defaults.LogSegmentBytes, atLeast(LegacyRecord.RECORD_OVERHEAD_V0), HIGH, LogSegmentBytesDoc) .define(LogRollTimeMillisProp, LONG, null, HIGH, LogRollTimeMillisDoc) .define(LogRollTimeHoursProp, INT, Defaults.LogRollHours, atLeast(1), HIGH, LogRollTimeHoursDoc) @@ -862,6 +864,7 @@ object KafkaConfig { .define(LogMessageTimestampDifferenceMaxMsProp, LONG, Defaults.LogMessageTimestampDifferenceMaxMs, MEDIUM, LogMessageTimestampDifferenceMaxMsDoc) .define(CreateTopicPolicyClassNameProp, CLASS, null, LOW, CreateTopicPolicyClassNameDoc) .define(AlterConfigPolicyClassNameProp, CLASS, null, LOW, AlterConfigPolicyClassNameDoc) + .define(LogMessageDownConversionEnableProp, BOOLEAN, Defaults.MessageDownConversionEnable, LOW, LogMessageDownConversionEnableDoc) /** ********* Replication configuration ***********/ .define(ControllerSocketTimeoutMsProp, INT, Defaults.ControllerSocketTimeoutMs, MEDIUM, ControllerSocketTimeoutMsDoc) @@ -1135,6 +1138,7 @@ class KafkaConfig(val props: java.util.Map[_, _], doLog: Boolean, dynamicConfigO val logMessageFormatVersion = ApiVersion(logMessageFormatVersionString) def logMessageTimestampType = TimestampType.forName(getString(KafkaConfig.LogMessageTimestampTypeProp)) def logMessageTimestampDifferenceMaxMs: Long = getLong(KafkaConfig.LogMessageTimestampDifferenceMaxMsProp) + def logMessageDownConversionEnable: Boolean = getBoolean(KafkaConfig.LogMessageDownConversionEnableProp) /** ********* Replication configuration ***********/ val controllerSocketTimeoutMs: Int = getInt(KafkaConfig.ControllerSocketTimeoutMsProp) diff --git a/core/src/main/scala/kafka/server/KafkaServer.scala b/core/src/main/scala/kafka/server/KafkaServer.scala index 57bca69743720..3d07e3672b35b 100755 --- a/core/src/main/scala/kafka/server/KafkaServer.scala +++ b/core/src/main/scala/kafka/server/KafkaServer.scala @@ -79,6 +79,7 @@ object KafkaServer { logProps.put(LogConfig.MessageFormatVersionProp, kafkaConfig.logMessageFormatVersion.version) logProps.put(LogConfig.MessageTimestampTypeProp, kafkaConfig.logMessageTimestampType.name) logProps.put(LogConfig.MessageTimestampDifferenceMaxMsProp, kafkaConfig.logMessageTimestampDifferenceMaxMs: java.lang.Long) + logProps.put(LogConfig.MessageDownConversionEnableProp, kafkaConfig.logMessageDownConversionEnable: java.lang.Boolean) logProps } @@ -615,6 +616,9 @@ class KafkaServer(val config: KafkaConfig, time: Time = Time.SYSTEM, threadNameP if (brokerTopicStats != null) CoreUtils.swallow(brokerTopicStats.close(), this) + // Clear all reconfigurable instances stored in DynamicBrokerConfig + config.dynamicConfig.clear() + brokerState.newState(NotRunning) startupComplete.set(false) diff --git a/core/src/main/scala/kafka/server/MetadataCache.scala b/core/src/main/scala/kafka/server/MetadataCache.scala index 43fe35287d384..b0603b87686b8 100755 --- a/core/src/main/scala/kafka/server/MetadataCache.scala +++ b/core/src/main/scala/kafka/server/MetadataCache.scala @@ -65,19 +65,29 @@ class MetadataCache(brokerId: Int) extends Logging { } // errorUnavailableEndpoints exists to support v0 MetadataResponses - private def getPartitionMetadata(topic: String, listenerName: ListenerName, errorUnavailableEndpoints: Boolean): Option[Iterable[MetadataResponse.PartitionMetadata]] = { + // If errorUnavailableListeners=true, return LISTENER_NOT_FOUND if listener is missing on the broker. + // Otherwise, return LEADER_NOT_AVAILABLE for broker unavailable and missing listener (Metadata response v5 and below). + private def getPartitionMetadata(topic: String, listenerName: ListenerName, errorUnavailableEndpoints: Boolean, + errorUnavailableListeners: Boolean): Option[Iterable[MetadataResponse.PartitionMetadata]] = { cache.get(topic).map { partitions => partitions.map { case (partitionId, partitionState) => val topicPartition = TopicAndPartition(topic, partitionId) - val maybeLeader = getAliveEndpoint(partitionState.basePartitionState.leader, listenerName) + val leaderBrokerId = partitionState.basePartitionState.leader + val maybeLeader = getAliveEndpoint(leaderBrokerId, listenerName) val replicas = partitionState.basePartitionState.replicas.asScala.map(_.toInt) val replicaInfo = getEndpoints(replicas, listenerName, errorUnavailableEndpoints) val offlineReplicaInfo = getEndpoints(partitionState.offlineReplicas.asScala.map(_.toInt), listenerName, errorUnavailableEndpoints) maybeLeader match { case None => - debug(s"Error while fetching metadata for $topicPartition: leader not available") - new MetadataResponse.PartitionMetadata(Errors.LEADER_NOT_AVAILABLE, partitionId, Node.noNode(), + val error = if (!aliveBrokers.contains(brokerId)) { // we are already holding the read lock + debug(s"Error while fetching metadata for $topicPartition: leader not available") + Errors.LEADER_NOT_AVAILABLE + } else { + debug(s"Error while fetching metadata for $topicPartition: listener $listenerName not found on leader $leaderBrokerId") + if (errorUnavailableListeners) Errors.LISTENER_NOT_FOUND else Errors.LEADER_NOT_AVAILABLE + } + new MetadataResponse.PartitionMetadata(error, partitionId, Node.noNode(), replicaInfo.asJava, java.util.Collections.emptyList(), offlineReplicaInfo.asJava) case Some(leader) => @@ -112,10 +122,11 @@ class MetadataCache(brokerId: Int) extends Logging { } // errorUnavailableEndpoints exists to support v0 MetadataResponses - def getTopicMetadata(topics: Set[String], listenerName: ListenerName, errorUnavailableEndpoints: Boolean = false): Seq[MetadataResponse.TopicMetadata] = { + def getTopicMetadata(topics: Set[String], listenerName: ListenerName, errorUnavailableEndpoints: Boolean = false, + errorUnavailableListeners: Boolean = false): Seq[MetadataResponse.TopicMetadata] = { inReadLock(partitionMetadataLock) { topics.toSeq.flatMap { topic => - getPartitionMetadata(topic, listenerName, errorUnavailableEndpoints).map { partitionMetadata => + getPartitionMetadata(topic, listenerName, errorUnavailableEndpoints, errorUnavailableListeners).map { partitionMetadata => new MetadataResponse.TopicMetadata(Errors.NONE, topic, Topic.isInternal(topic), partitionMetadata.toBuffer.asJava) } } diff --git a/core/src/main/scala/kafka/server/QuotaFactory.scala b/core/src/main/scala/kafka/server/QuotaFactory.scala index 1ee713b10bef3..ed04dcf096fb8 100644 --- a/core/src/main/scala/kafka/server/QuotaFactory.scala +++ b/core/src/main/scala/kafka/server/QuotaFactory.scala @@ -37,7 +37,8 @@ object QuotaFactory extends Logging { object UnboundedQuota extends ReplicaQuota { override def isThrottled(topicPartition: TopicPartition): Boolean = false - override def isQuotaExceeded(): Boolean = false + override def isQuotaExceeded: Boolean = false + def record(value: Long): Unit = () } case class QuotaManagers(fetch: ClientQuotaManager, diff --git a/core/src/main/scala/kafka/server/ReplicaAlterLogDirsThread.scala b/core/src/main/scala/kafka/server/ReplicaAlterLogDirsThread.scala index 5a505c3d3779c..59a8b8ed6be13 100644 --- a/core/src/main/scala/kafka/server/ReplicaAlterLogDirsThread.scala +++ b/core/src/main/scala/kafka/server/ReplicaAlterLogDirsThread.scala @@ -25,7 +25,7 @@ import kafka.cluster.BrokerEndPoint import kafka.server.AbstractFetcherThread.ResultWithPartitions import kafka.server.QuotaFactory.UnboundedQuota import kafka.server.ReplicaAlterLogDirsThread.{FetchRequest, PartitionData} -import kafka.server.epoch.LeaderEpochCache +import kafka.server.epoch.LeaderEpochFileCache import org.apache.kafka.common.TopicPartition import org.apache.kafka.common.errors.KafkaStorageException import org.apache.kafka.common.protocol.{ApiKeys, Errors} @@ -98,8 +98,7 @@ class ReplicaAlterLogDirsThread(name: String, throw new IllegalStateException("Offset mismatch for the future replica %s: fetched offset = %d, log end offset = %d.".format( topicPartition, fetchOffset, futureReplica.logEndOffset.messageOffset)) - // Append the leader's messages to the log - partition.appendRecordsToFutureReplica(records) + partition.appendRecordsToFollowerOrFutureReplica(records, isFuture = true) val futureReplicaHighWatermark = futureReplica.logEndOffset.messageOffset.min(partitionData.highWatermark) futureReplica.highWatermark = new LogOffsetMetadata(futureReplicaHighWatermark) futureReplica.maybeIncrementLogStartOffset(partitionData.logStartOffset) @@ -142,17 +141,19 @@ class ReplicaAlterLogDirsThread(name: String, * Builds offset for leader epoch requests for partitions that are in the truncating phase based * on latest epochs of the future replicas (the one that is fetching) */ - def buildLeaderEpochRequest(allPartitions: Seq[(TopicPartition, PartitionFetchState)]): ResultWithPartitions[Map[TopicPartition, Int]] = { - def epochCacheOpt(tp: TopicPartition): Option[LeaderEpochCache] = replicaMgr.getReplica(tp, Request.FutureLocalReplicaId).map(_.epochs.get) + def buildLeaderEpochRequest(allPartitions: Seq[(TopicPartition, PartitionFetchState)]): (Map[TopicPartition, Int], Set[TopicPartition]) = { + def epochCacheOpt(tp: TopicPartition): Option[LeaderEpochFileCache] = replicaMgr.getReplica(tp, Request.FutureLocalReplicaId).flatMap(_.epochs) val partitionEpochOpts = allPartitions .filter { case (_, state) => state.isTruncatingLog } .map { case (tp, _) => tp -> epochCacheOpt(tp) }.toMap - val (partitionsWithEpoch, partitionsWithoutEpoch) = partitionEpochOpts.partition { case (_, epochCacheOpt) => epochCacheOpt.nonEmpty } + val (partitionsWithEpoch, partitionsWithoutEpoch) = partitionEpochOpts.partition { case (_, epochCacheOpt) => + epochCacheOpt.exists(_.latestEpoch != UNDEFINED_EPOCH) + } - val result = partitionsWithEpoch.map { case (tp, epochCacheOpt) => tp -> epochCacheOpt.get.latestEpoch() } - ResultWithPartitions(result, partitionsWithoutEpoch.keys.toSet) + val result = partitionsWithEpoch.map { case (tp, epochCacheOpt) => tp -> epochCacheOpt.get.latestEpoch } + (result, partitionsWithoutEpoch.keys.toSet) } /** @@ -163,7 +164,7 @@ class ReplicaAlterLogDirsThread(name: String, def fetchEpochsFromLeader(partitions: Map[TopicPartition, Int]): Map[TopicPartition, EpochEndOffset] = { partitions.map { case (tp, epoch) => try { - val (leaderEpoch, leaderOffset) = replicaMgr.getReplicaOrException(tp).epochs.get.endOffsetFor(epoch) + val (leaderEpoch, leaderOffset) = replicaMgr.getReplicaOrException(tp).endOffsetFor(epoch) tp -> new EpochEndOffset(Errors.NONE, leaderEpoch, leaderOffset) } catch { case t: Throwable => diff --git a/core/src/main/scala/kafka/server/ReplicaFetcherBlockingSend.scala b/core/src/main/scala/kafka/server/ReplicaFetcherBlockingSend.scala index 0bf2bd3c9fff7..6048c669f6303 100644 --- a/core/src/main/scala/kafka/server/ReplicaFetcherBlockingSend.scala +++ b/core/src/main/scala/kafka/server/ReplicaFetcherBlockingSend.scala @@ -35,6 +35,8 @@ trait BlockingSend { def sendRequest(requestBuilder: AbstractRequest.Builder[_ <: AbstractRequest]): ClientResponse + def initiateClose() + def close() } @@ -86,7 +88,7 @@ class ReplicaFetcherBlockingSend(sourceBroker: BrokerEndPoint, ) } - override def sendRequest(requestBuilder: Builder[_ <: AbstractRequest]): ClientResponse = { + override def sendRequest(requestBuilder: Builder[_ <: AbstractRequest]): ClientResponse = { try { if (!NetworkClientUtils.awaitReady(networkClient, sourceNode, time, socketTimeout)) throw new SocketTimeoutException(s"Failed to connect within $socketTimeout ms") @@ -103,6 +105,10 @@ class ReplicaFetcherBlockingSend(sourceBroker: BrokerEndPoint, } } + override def initiateClose(): Unit = { + networkClient.initiateClose() + } + def close(): Unit = { networkClient.close() } diff --git a/core/src/main/scala/kafka/server/ReplicaFetcherThread.scala b/core/src/main/scala/kafka/server/ReplicaFetcherThread.scala index cf8d829f85077..94c26dee5a606 100644 --- a/core/src/main/scala/kafka/server/ReplicaFetcherThread.scala +++ b/core/src/main/scala/kafka/server/ReplicaFetcherThread.scala @@ -20,17 +20,14 @@ package kafka.server import java.util import AbstractFetcherThread.ResultWithPartitions -import kafka.api.{FetchRequest => _, _} +import kafka.api._ import kafka.cluster.BrokerEndPoint -import kafka.log.LogConfig import kafka.server.ReplicaFetcherThread._ -import kafka.server.epoch.LeaderEpochCache -import kafka.zk.AdminZkClient +import kafka.server.epoch.LeaderEpochFileCache import org.apache.kafka.clients.FetchSessionHandler import org.apache.kafka.common.requests.EpochEndOffset._ import org.apache.kafka.common.TopicPartition import org.apache.kafka.common.errors.KafkaStorageException -import org.apache.kafka.common.internals.FatalExitError import org.apache.kafka.common.metrics.Metrics import org.apache.kafka.common.protocol.Errors import org.apache.kafka.common.record.{MemoryRecords, Records} @@ -47,7 +44,7 @@ class ReplicaFetcherThread(name: String, replicaMgr: ReplicaManager, metrics: Metrics, time: Time, - quota: ReplicationQuotaManager, + quota: ReplicaQuota, leaderEndpointBlockingSend: Option[BlockingSend] = None) extends AbstractFetcherThread(name = name, clientId = name, @@ -63,20 +60,34 @@ class ReplicaFetcherThread(name: String, private val logContext = new LogContext(s"[ReplicaFetcher replicaId=$replicaId, leaderId=${sourceBroker.id}, " + s"fetcherId=$fetcherId] ") this.logIdent = logContext.logPrefix + private val leaderEndpoint = leaderEndpointBlockingSend.getOrElse( new ReplicaFetcherBlockingSend(sourceBroker, brokerConfig, metrics, time, fetcherId, s"broker-$replicaId-fetcher-$fetcherId", logContext)) - private val fetchRequestVersion: Short = - if (brokerConfig.interBrokerProtocolVersion >= KAFKA_1_1_IV0) 7 + + // Visible for testing + private[server] val fetchRequestVersion: Short = + if (brokerConfig.interBrokerProtocolVersion >= KAFKA_2_0_IV1) 8 + else if (brokerConfig.interBrokerProtocolVersion >= KAFKA_1_1_IV0) 7 else if (brokerConfig.interBrokerProtocolVersion >= KAFKA_0_11_0_IV1) 5 else if (brokerConfig.interBrokerProtocolVersion >= KAFKA_0_11_0_IV0) 4 else if (brokerConfig.interBrokerProtocolVersion >= KAFKA_0_10_1_IV1) 3 else if (brokerConfig.interBrokerProtocolVersion >= KAFKA_0_10_0_IV0) 2 else if (brokerConfig.interBrokerProtocolVersion >= KAFKA_0_9_0) 1 else 0 - private val offsetForLeaderEpochRequestVersion: Short = + + // Visible for testing + private[server] val offsetForLeaderEpochRequestVersion: Short = if (brokerConfig.interBrokerProtocolVersion >= KAFKA_2_0_IV0) 1 else 0 + + // Visible for testing + private[server] val listOffsetRequestVersion: Short = + if (brokerConfig.interBrokerProtocolVersion >= KAFKA_2_0_IV1) 3 + else if (brokerConfig.interBrokerProtocolVersion >= KAFKA_0_11_0_IV0) 2 + else if (brokerConfig.interBrokerProtocolVersion >= KAFKA_0_10_1_IV2) 1 + else 0 + private val fetchMetadataSupported = brokerConfig.interBrokerProtocolVersion >= KAFKA_1_1_IV0 private val maxWait = brokerConfig.replicaFetchWaitMaxMs private val minBytes = brokerConfig.replicaFetchMinBytes @@ -85,16 +96,37 @@ class ReplicaFetcherThread(name: String, private val shouldSendLeaderEpochRequest: Boolean = brokerConfig.interBrokerProtocolVersion >= KAFKA_0_11_0_IV2 private val fetchSessionHandler = new FetchSessionHandler(logContext, sourceBroker.id) - private def epochCacheOpt(tp: TopicPartition): Option[LeaderEpochCache] = replicaMgr.getReplica(tp).map(_.epochs.get) + private def epochCacheOpt(tp: TopicPartition): Option[LeaderEpochFileCache] = replicaMgr.getReplica(tp).flatMap(_.epochs) override def initiateShutdown(): Boolean = { val justShutdown = super.initiateShutdown() if (justShutdown) { - leaderEndpoint.close() + // This is thread-safe, so we don't expect any exceptions, but catch and log any errors + // to avoid failing the caller, especially during shutdown. We will attempt to close + // leaderEndpoint after the thread terminates. + try { + leaderEndpoint.initiateClose() + } catch { + case t: Throwable => + error(s"Failed to initiate shutdown of leader endpoint $leaderEndpoint after initiating replica fetcher thread shutdown", t) + } } justShutdown } + override def awaitShutdown(): Unit = { + super.awaitShutdown() + // We don't expect any exceptions here, but catch and log any errors to avoid failing the caller, + // especially during shutdown. It is safe to catch the exception here without causing correctness + // issue because we are going to shutdown the thread and will not re-use the leaderEndpoint anyway. + try { + leaderEndpoint.close() + } catch { + case t: Throwable => + error(s"Failed to close leader endpoint $leaderEndpoint after shutting down replica fetcher thread", t) + } + } + // process fetched data def processPartitionData(topicPartition: TopicPartition, fetchOffset: Long, partitionData: PartitionData) { val replica = replicaMgr.getReplicaOrException(topicPartition) @@ -112,7 +144,7 @@ class ReplicaFetcherThread(name: String, .format(replica.logEndOffset.messageOffset, topicPartition, records.sizeInBytes, partitionData.highWatermark)) // Append the leader's messages to the log - partition.appendRecordsToFollower(records) + partition.appendRecordsToFollowerOrFutureReplica(records, isFuture = false) if (isTraceEnabled) trace("Follower has replica log end offset %d after appending %d bytes of messages for partition %s" @@ -163,18 +195,6 @@ class ReplicaFetcherThread(name: String, val leaderEndOffset: Long = earliestOrLatestOffset(topicPartition, ListOffsetRequest.LATEST_TIMESTAMP) 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. - val adminZkClient = new AdminZkClient(replicaMgr.zkClient) - if (!LogConfig.fromProps(brokerConfig.originals, adminZkClient.fetchEntityConfig( - ConfigType.Topic, topicPartition.topic)).uncleanLeaderElectionEnable) { - // Log a fatal error and shutdown the broker to ensure that data loss does not occur unexpectedly. - fatal(s"Exiting because log truncation is not allowed for partition $topicPartition, current leader's " + - s"latest offset $leaderEndOffset is less than replica's latest offset ${replica.logEndOffset.messageOffset}") - throw new FatalExitError - } - warn(s"Reset fetch offset for partition $topicPartition from ${replica.logEndOffset.messageOffset} to current " + s"leader's latest offset $leaderEndOffset") partition.truncateTo(leaderEndOffset, isFuture = false) @@ -242,10 +262,10 @@ class ReplicaFetcherThread(name: String, private def earliestOrLatestOffset(topicPartition: TopicPartition, earliestOrLatest: Long): Long = { val requestBuilder = if (brokerConfig.interBrokerProtocolVersion >= KAFKA_0_10_1_IV2) { val partitions = Map(topicPartition -> (earliestOrLatest: java.lang.Long)) - ListOffsetRequest.Builder.forReplica(1, replicaId).setTargetTimes(partitions.asJava) + ListOffsetRequest.Builder.forReplica(listOffsetRequestVersion, replicaId).setTargetTimes(partitions.asJava) } else { val partitions = Map(topicPartition -> new ListOffsetRequest.PartitionData(earliestOrLatest, 1)) - ListOffsetRequest.Builder.forReplica(0, replicaId).setOffsetData(partitions.asJava) + ListOffsetRequest.Builder.forReplica(listOffsetRequestVersion, replicaId).setOffsetData(partitions.asJava) } val clientResponse = leaderEndpoint.sendRequest(requestBuilder) val response = clientResponse.responseBody.asInstanceOf[ListOffsetResponse] @@ -328,16 +348,18 @@ class ReplicaFetcherThread(name: String, ResultWithPartitions(fetchOffsets, partitionsWithError) } - override def buildLeaderEpochRequest(allPartitions: Seq[(TopicPartition, PartitionFetchState)]): ResultWithPartitions[Map[TopicPartition, Int]] = { + override def buildLeaderEpochRequest(allPartitions: Seq[(TopicPartition, PartitionFetchState)]): (Map[TopicPartition, Int], Set[TopicPartition]) = { val partitionEpochOpts = allPartitions .filter { case (_, state) => state.isTruncatingLog } .map { case (tp, _) => tp -> epochCacheOpt(tp) }.toMap - val (partitionsWithEpoch, partitionsWithoutEpoch) = partitionEpochOpts.partition { case (_, epochCacheOpt) => epochCacheOpt.nonEmpty } + val (partitionsWithEpoch, partitionsWithoutEpoch) = partitionEpochOpts.partition { case (_, epochCacheOpt) => + epochCacheOpt.exists(_.latestEpoch != UNDEFINED_EPOCH) + } debug(s"Build leaderEpoch request $partitionsWithEpoch") - val result = partitionsWithEpoch.map { case (tp, epochCacheOpt) => tp -> epochCacheOpt.get.latestEpoch() } - ResultWithPartitions(result, partitionsWithoutEpoch.keys.toSet) + val result = partitionsWithEpoch.map { case (tp, epochCacheOpt) => tp -> epochCacheOpt.get.latestEpoch } + (result, partitionsWithoutEpoch.keys.toSet) } override def fetchEpochsFromLeader(partitions: Map[TopicPartition, Int]): Map[TopicPartition, EpochEndOffset] = { diff --git a/core/src/main/scala/kafka/server/ReplicaManager.scala b/core/src/main/scala/kafka/server/ReplicaManager.scala index 24f3235570f23..5e52f9a4603ae 100644 --- a/core/src/main/scala/kafka/server/ReplicaManager.scala +++ b/core/src/main/scala/kafka/server/ReplicaManager.scala @@ -25,7 +25,7 @@ import com.yammer.metrics.core.Gauge import kafka.api._ import kafka.cluster.{BrokerEndPoint, Partition, Replica} import kafka.controller.{KafkaController, StateChangeLogger} -import kafka.log.{Log, LogAppendInfo, LogManager} +import kafka.log.{Log, LogAppendInfo, LogConfig, LogManager} import kafka.metrics.KafkaMetricsGroup import kafka.server.QuotaFactory.{QuotaManagers, UnboundedQuota} import kafka.server.checkpoints.OffsetCheckpointFile @@ -90,11 +90,6 @@ case class LogReadResult(info: FetchDataInfo, case Some(e) => Errors.forException(e) } - def updateLeaderReplicaInfo(leaderReplica: Replica): LogReadResult = - copy(highWatermark = leaderReplica.highWatermark.messageOffset, - leaderLogStartOffset = leaderReplica.logStartOffset, - leaderLogEndOffset = leaderReplica.logEndOffset.messageOffset) - def withEmptyFetchInfo: LogReadResult = copy(info = FetchDataInfo(LogOffsetMetadata.UnknownOffsetMetadata, MemoryRecords.EMPTY)) @@ -335,8 +330,10 @@ class ReplicaManager(val config: KafkaConfig, if (deletePartition) { val removedPartition = allPartitions.remove(topicPartition) - if (removedPartition eq ReplicaManager.OfflinePartition) + if (removedPartition eq ReplicaManager.OfflinePartition) { + allPartitions.put(topicPartition, ReplicaManager.OfflinePartition) throw new KafkaStorageException(s"Partition $topicPartition is on an offline disk") + } if (removedPartition != null) { val topicHasPartitions = allPartitions.values.exists(partition => topicPartition.topic == partition.topic) @@ -577,14 +574,17 @@ class ReplicaManager(val config: KafkaConfig, if (!logManager.isLogDirOnline(destinationDir)) throw new KafkaStorageException(s"Log directory $destinationDir is offline") - // Stop current replica movement if the destinationDir is different from the existing destination log directory - getReplica(topicPartition, Request.FutureLocalReplicaId) match { - case Some(futureReplica) => - if (futureReplica.log.get.dir.getParent != destinationDir) { + getPartition(topicPartition) match { + case Some(partition) => + if (partition eq ReplicaManager.OfflinePartition) + throw new KafkaStorageException(s"Partition $topicPartition is offline") + + // Stop current replica movement if the destinationDir is different from the existing destination log directory + if (partition.futureReplicaDirChanged(destinationDir)) { replicaAlterLogDirsManager.removeFetcherForPartitions(Set(topicPartition)) - getPartition(topicPartition).get.removeFutureLocalReplica() - logManager.asyncDelete(topicPartition, isFuture = true) + partition.removeFutureLocalReplica() } + case None => } @@ -995,8 +995,9 @@ class ReplicaManager(val config: KafkaConfig, quota.isThrottled(topicPartition) && quota.isQuotaExceeded && !isReplicaInSync } - def getMagic(topicPartition: TopicPartition): Option[Byte] = - getReplica(topicPartition).flatMap(_.log.map(_.config.messageFormatVersion.recordVersion.value)) + def getLogConfig(topicPartition: TopicPartition): Option[LogConfig] = getReplica(topicPartition).flatMap(_.log.map(_.config)) + + def getMagic(topicPartition: TopicPartition): Option[Byte] = getLogConfig(topicPartition).map(_.messageFormatVersion.recordVersion.value) def maybeUpdateMetadataCache(correlationId: Int, updateMetadataRequest: UpdateMetadataRequest) : Seq[TopicPartition] = { replicaStateChangeLock synchronized { @@ -1336,7 +1337,12 @@ class ReplicaManager(val config: KafkaConfig, /** * Update the follower's fetch state in the leader based on the last fetch request and update `readResult`, - * if necessary. + * if the follower replica is not recognized to be one of the assigned replicas. Do not update + * `readResult` otherwise, so that log start/end offset and high watermark is consistent with + * records in fetch response. Log start/end offset and high watermark may change not only due to + * this fetch request, e.g., rolling new log segment and removing old log segment may move log + * start offset further than the last offset in the fetched records. The followers will get the + * updated leader's state in the next fetch response. */ private def updateFollowerLogReadResults(replicaId: Int, readResults: Seq[(TopicPartition, LogReadResult)]): Seq[(TopicPartition, LogReadResult)] = { @@ -1347,10 +1353,7 @@ class ReplicaManager(val config: KafkaConfig, case Some(partition) => partition.getReplica(replicaId) match { case Some(replica) => - if (partition.updateReplicaLogReadResult(replica, readResult)) - partition.leaderReplicaIfLocal.foreach { leaderReplica => - updatedReadResult = readResult.updateLeaderReplicaInfo(leaderReplica) - } + partition.updateReplicaLogReadResult(replica, readResult) case None => warn(s"Leader $localBrokerId failed to record follower $replicaId's position " + s"${readResult.info.fetchOffsetMetadata.messageOffset} since the replica is not recognized to be " + @@ -1397,7 +1400,8 @@ class ReplicaManager(val config: KafkaConfig, } // logDir should be an absolute path - def handleLogDirFailure(dir: String) { + // sendZkNotification is needed for unit test + def handleLogDirFailure(dir: String, sendZkNotification: Boolean = true) { if (!logManager.isLogDirOnline(dir)) return info(s"Stopping serving replicas in dir $dir") @@ -1417,7 +1421,7 @@ class ReplicaManager(val config: KafkaConfig, replicaFetcherManager.removeFetcherForPartitions(newOfflinePartitions) replicaAlterLogDirsManager.removeFetcherForPartitions(newOfflinePartitions ++ partitionsWithOfflineFutureReplica.map(_.topicPartition)) - partitionsWithOfflineFutureReplica.foreach(partition => partition.removeFutureLocalReplica()) + partitionsWithOfflineFutureReplica.foreach(partition => partition.removeFutureLocalReplica(deleteFromLogDir = false)) newOfflinePartitions.foreach { topicPartition => val partition = allPartitions.put(topicPartition, ReplicaManager.OfflinePartition) partition.removePartitionMetrics() @@ -1433,7 +1437,9 @@ class ReplicaManager(val config: KafkaConfig, s"for partitions ${partitionsWithOfflineFutureReplica.mkString(",")} because they are in the failed log directory $dir.") } logManager.handleLogDirFailure(dir) - zkClient.propagateLogDirEvent(localBrokerId) + + if (sendZkNotification) + zkClient.propagateLogDirEvent(localBrokerId) info(s"Stopped serving replicas in dir $dir") } diff --git a/core/src/main/scala/kafka/server/ReplicationQuotaManager.scala b/core/src/main/scala/kafka/server/ReplicationQuotaManager.scala index 84004e3f8e66a..7835c9dbafe4e 100644 --- a/core/src/main/scala/kafka/server/ReplicationQuotaManager.scala +++ b/core/src/main/scala/kafka/server/ReplicationQuotaManager.scala @@ -51,8 +51,9 @@ object ReplicationQuotaManagerConfig { } trait ReplicaQuota { + def record(value: Long): Unit def isThrottled(topicPartition: TopicPartition): Boolean - def isQuotaExceeded(): Boolean + def isQuotaExceeded: Boolean } object Constants { @@ -99,7 +100,7 @@ class ReplicationQuotaManager(val config: ReplicationQuotaManagerConfig, * * @return */ - override def isQuotaExceeded(): Boolean = { + override def isQuotaExceeded: Boolean = { try { sensor().checkQuotas() } catch { diff --git a/core/src/main/scala/kafka/server/epoch/LeaderEpochFileCache.scala b/core/src/main/scala/kafka/server/epoch/LeaderEpochFileCache.scala index 23a53056f3209..9e7077dfe4b8e 100644 --- a/core/src/main/scala/kafka/server/epoch/LeaderEpochFileCache.scala +++ b/core/src/main/scala/kafka/server/epoch/LeaderEpochFileCache.scala @@ -18,63 +18,83 @@ package kafka.server.epoch import java.util.concurrent.locks.ReentrantReadWriteLock -import kafka.server.LogOffsetMetadata import kafka.server.checkpoints.LeaderEpochCheckpoint import org.apache.kafka.common.requests.EpochEndOffset._ import kafka.utils.CoreUtils._ import kafka.utils.Logging import org.apache.kafka.common.TopicPartition -import scala.collection.mutable.ListBuffer -trait LeaderEpochCache { - def assign(leaderEpoch: Int, offset: Long) - def latestEpoch(): Int - def endOffsetFor(epoch: Int): (Int, Long) - def clearAndFlushLatest(offset: Long) - def clearAndFlushEarliest(offset: Long) - def clearAndFlush() - def clear() -} +import scala.collection.mutable.ListBuffer /** - * Represents a cache of (LeaderEpoch => Offset) mappings for a particular replica. - * - * Leader Epoch = epoch assigned to each leader by the controller. - * Offset = offset of the first message in each epoch. - * - * @param leo a function that determines the log end offset - * @param checkpoint the checkpoint file - */ -class LeaderEpochFileCache(topicPartition: TopicPartition, leo: () => LogOffsetMetadata, checkpoint: LeaderEpochCheckpoint) extends LeaderEpochCache with Logging { + * Represents a cache of (LeaderEpoch => Offset) mappings for a particular replica. + * + * Leader Epoch = epoch assigned to each leader by the controller. + * Offset = offset of the first message in each epoch. + * + * @param topicPartition the associated topic partition + * @param checkpoint the checkpoint file + * @param logEndOffset function to fetch the current log end offset + */ +class LeaderEpochFileCache(topicPartition: TopicPartition, + logEndOffset: () => Long, + checkpoint: LeaderEpochCheckpoint) extends Logging { + this.logIdent = s"[LeaderEpochCache $topicPartition] " + private val lock = new ReentrantReadWriteLock() private var epochs: ListBuffer[EpochEntry] = inWriteLock(lock) { ListBuffer(checkpoint.read(): _*) } /** * Assigns the supplied Leader Epoch to the supplied Offset * Once the epoch is assigned it cannot be reassigned - * - * @param epoch - * @param offset */ - override def assign(epoch: Int, offset: Long): Unit = { + def assign(epoch: Int, startOffset: Long): Unit = { inWriteLock(lock) { - if (epoch >= 0 && epoch > latestEpoch && offset >= latestOffset) { - info(s"Updated PartitionLeaderEpoch. ${epochChangeMsg(epoch, offset)}. Cache now contains ${epochs.size} entries.") - epochs += EpochEntry(epoch, offset) - flush() + val updateNeeded = if (epochs.isEmpty) { + true } else { - validateAndMaybeWarn(epoch, offset) + val lastEntry = epochs.last + lastEntry.epoch != epoch || startOffset < lastEntry.startOffset } + + if (updateNeeded) { + truncateAndAppend(EpochEntry(epoch, startOffset)) + flush() + } + } + } + + /** + * Remove any entries which violate monotonicity following the insertion of an assigned epoch. + */ + private def truncateAndAppend(entryToAppend: EpochEntry): Unit = { + validateAndMaybeWarn(entryToAppend) + + val (retainedEpochs, removedEpochs) = epochs.partition { entry => + entry.epoch < entryToAppend.epoch && entry.startOffset < entryToAppend.startOffset + } + + epochs = retainedEpochs :+ entryToAppend + + if (removedEpochs.isEmpty) { + debug(s"Appended new epoch entry $entryToAppend. Cache now contains ${epochs.size} entries.") + } else { + warn(s"New epoch entry $entryToAppend caused truncation of conflicting entries $removedEpochs. " + + s"Cache now contains ${epochs.size} entries.") } } + def nonEmpty: Boolean = inReadLock(lock) { + epochs.nonEmpty + } + /** * Returns the current Leader Epoch. This is the latest epoch * which has messages assigned to it. * * @return */ - override def latestEpoch(): Int = { + def latestEpoch: Int = { inReadLock(lock) { if (epochs.isEmpty) UNDEFINED_EPOCH else epochs.last.epoch } @@ -93,45 +113,59 @@ class LeaderEpochFileCache(topicPartition: TopicPartition, leo: () => LogOffsetM * so that the follower falls back to High Water Mark. * * @param requestedEpoch requested leader epoch - * @return leader epoch and offset + * @return found leader epoch and end offset */ - override def endOffsetFor(requestedEpoch: Int): (Int, Long) = { + def endOffsetFor(requestedEpoch: Int): (Int, Long) = { inReadLock(lock) { val epochAndOffset = if (requestedEpoch == UNDEFINED_EPOCH) { - // this may happen if a bootstrapping follower sends a request with undefined epoch or + // This may happen if a bootstrapping follower sends a request with undefined epoch or // a follower is on the older message format where leader epochs are not recorded (UNDEFINED_EPOCH, UNDEFINED_EPOCH_OFFSET) } else if (requestedEpoch == latestEpoch) { - (requestedEpoch, leo().messageOffset) + // For the leader, the latest epoch is always the current leader epoch that is still being written to. + // Followers should not have any reason to query for the end offset of the current epoch, but a consumer + // might if it is verifying its committed offset following a group rebalance. In this case, we return + // the current log end offset which makes the truncation check work as expected. + (requestedEpoch, logEndOffset()) } else { val (subsequentEpochs, previousEpochs) = epochs.partition { e => e.epoch > requestedEpoch} - if (subsequentEpochs.isEmpty || requestedEpoch < epochs.head.epoch) - // no epochs recorded or requested epoch < the first epoch cached + if (subsequentEpochs.isEmpty) { + // The requested epoch is larger than any known epoch. This case should never be hit because + // the latest cached epoch is always the largest. (UNDEFINED_EPOCH, UNDEFINED_EPOCH_OFFSET) - else { - // we must get at least one element in previous epochs list, because if we are here, - // it means that requestedEpoch >= epochs.head.epoch -- so at least the first epoch is + } else if (previousEpochs.isEmpty) { + // The requested epoch is smaller than any known epoch, so we return the start offset of the first + // known epoch which is larger than it. This may be inaccurate as there could have been + // epochs in between, but the point is that the data has already been removed from the log + // and we want to ensure that the follower can replicate correctly beginning from the leader's + // start offset. + (requestedEpoch, subsequentEpochs.head.startOffset) + } else { + // We have at least one previous epoch and one subsequent epoch. The result is the first + // prior epoch and the starting offset of the first subsequent epoch. (previousEpochs.last.epoch, subsequentEpochs.head.startOffset) } } - debug(s"Processed offset for epoch request for partition ${topicPartition} epoch:$requestedEpoch and returning epoch ${epochAndOffset._1} and offset ${epochAndOffset._2} from epoch list of size ${epochs.size}") + debug(s"Processed end offset request for epoch $requestedEpoch and returning epoch ${epochAndOffset._1} " + + s"with end offset ${epochAndOffset._2} from epoch cache of size ${epochs.size}") epochAndOffset } } /** * Removes all epoch entries from the store with start offsets greater than or equal to the passed offset. - * - * @param offset */ - override def clearAndFlushLatest(offset: Long): Unit = { + def truncateFromEnd(endOffset: Long): Unit = { inWriteLock(lock) { - val before = epochs - if (offset >= 0 && offset <= latestOffset()) { - epochs = epochs.filter(entry => entry.startOffset < offset) + if (endOffset >= 0 && latestEntry.exists(_.startOffset >= endOffset)) { + val (subsequentEntries, previousEntries) = epochs.partition(_.startOffset >= endOffset) + epochs = previousEntries + flush() - info(s"Cleared latest ${before.toSet.filterNot(epochs.toSet)} entries from epoch cache based on passed offset $offset leaving ${epochs.size} in EpochFile for partition $topicPartition") + + debug(s"Cleared entries $subsequentEntries from epoch cache after " + + s"truncating to end offset $endOffset, leaving ${epochs.size} entries in the cache.") } } } @@ -142,20 +176,21 @@ class LeaderEpochFileCache(topicPartition: TopicPartition, leo: () => LogOffsetM * * This method is exclusive: so clearEarliest(6) will retain an entry at offset 6. * - * @param offset the offset to clear up to + * @param startOffset the offset to clear up to */ - override def clearAndFlushEarliest(offset: Long): Unit = { + def truncateFromStart(startOffset: Long): Unit = { inWriteLock(lock) { - val before = epochs - if (offset >= 0 && earliestOffset() < offset) { - val earliest = epochs.filter(entry => entry.startOffset < offset) - if (earliest.nonEmpty) { - epochs = epochs --= earliest - //If the offset is less than the earliest offset remaining, add previous epoch back, but with an updated offset - if (offset < earliestOffset() || epochs.isEmpty) - new EpochEntry(earliest.last.epoch, offset) +=: epochs + if (epochs.nonEmpty) { + val (subsequentEntries, previousEntries) = epochs.partition(_.startOffset > startOffset) + + previousEntries.lastOption.foreach { firstBeforeStartOffset => + val updatedFirstEntry = EpochEntry(firstBeforeStartOffset.epoch, startOffset) + epochs = updatedFirstEntry +: subsequentEntries + flush() - info(s"Cleared earliest ${before.toSet.filterNot(epochs.toSet).size} entries from epoch cache based on passed offset $offset leaving ${epochs.size} in EpochFile for partition $topicPartition") + + debug(s"Cleared entries $previousEntries and rewrote first entry $updatedFirstEntry after " + + s"truncating to start offset $startOffset, leaving ${epochs.size} in the cache.") } } } @@ -164,47 +199,55 @@ class LeaderEpochFileCache(topicPartition: TopicPartition, leo: () => LogOffsetM /** * Delete all entries. */ - override def clearAndFlush() = { + def clearAndFlush() = { inWriteLock(lock) { epochs.clear() flush() } } - override def clear() = { + def clear() = { inWriteLock(lock) { epochs.clear() } } - def epochEntries(): ListBuffer[EpochEntry] = { + // Visible for testing + def epochEntries: ListBuffer[EpochEntry] = { epochs } - private def earliestOffset(): Long = { - if (epochs.isEmpty) -1 else epochs.head.startOffset - } - - private def latestOffset(): Long = { - if (epochs.isEmpty) -1 else epochs.last.startOffset - } + private def latestEntry: Option[EpochEntry] = epochs.lastOption private def flush(): Unit = { checkpoint.write(epochs) } - def epochChangeMsg(epoch: Int, offset: Long) = s"New: {epoch:$epoch, offset:$offset}, Current: {epoch:$latestEpoch, offset:$latestOffset} for Partition: $topicPartition" - - def validateAndMaybeWarn(epoch: Int, offset: Long) = { - assert(epoch >= 0, s"Received a PartitionLeaderEpoch assignment for an epoch < 0. This should not happen. ${epochChangeMsg(epoch, offset)}") - if (epoch < latestEpoch()) - warn(s"Received a PartitionLeaderEpoch assignment for an epoch < latestEpoch. " + - s"This implies messages have arrived out of order. ${epochChangeMsg(epoch, offset)}") - else if (offset < latestOffset()) - warn(s"Received a PartitionLeaderEpoch assignment for an offset < latest offset for the most recent, stored PartitionLeaderEpoch. " + - s"This implies messages have arrived out of order. ${epochChangeMsg(epoch, offset)}") + private def validateAndMaybeWarn(entry: EpochEntry) = { + if (entry.epoch < 0) { + throw new IllegalArgumentException(s"Received invalid partition leader epoch entry $entry") + } else { + // If the latest append violates the monotonicity of epochs or starting offsets, our choices + // are either to raise an error, ignore the append, or allow the append and truncate the + // conflicting entries from the cache. Raising an error risks killing the fetcher threads in + // pathological cases (i.e. cases we are not yet aware of). We instead take the final approach + // and assume that the latest append is always accurate. + + latestEntry.foreach { latest => + if (entry.epoch < latest.epoch) + warn(s"Received leader epoch assignment $entry which has an epoch less than the epoch " + + s"of the latest entry $latest. This implies messages have arrived out of order.") + else if (entry.startOffset < latest.startOffset) + warn(s"Received leader epoch assignment $entry which has a starting offset which is less than " + + s"the starting offset of the latest entry $latest. This implies messages have arrived out of order.") + } + } } } // Mapping of epoch to the first offset of the subsequent epoch -case class EpochEntry(epoch: Int, startOffset: Long) +case class EpochEntry(epoch: Int, startOffset: Long) { + override def toString: String = { + s"EpochEntry(epoch=$epoch, startOffset=$startOffset)" + } +} diff --git a/core/src/main/scala/kafka/tools/ConsoleConsumer.scala b/core/src/main/scala/kafka/tools/ConsoleConsumer.scala index b3103ebfa8851..c55f6c484c754 100755 --- a/core/src/main/scala/kafka/tools/ConsoleConsumer.scala +++ b/core/src/main/scala/kafka/tools/ConsoleConsumer.scala @@ -20,20 +20,19 @@ package kafka.tools import java.io.PrintStream import java.nio.charset.StandardCharsets import java.util.concurrent.CountDownLatch -import java.util.{Locale, Properties, Random} +import java.util.regex.Pattern +import java.util.{Collections, Locale, Properties, Random} import com.typesafe.scalalogging.LazyLogging import joptsimple._ -import kafka.api.OffsetRequest -import kafka.common.{MessageFormatter, StreamEndException} -import kafka.consumer._ -import kafka.message._ -import kafka.metrics.KafkaMetricsReporter +import kafka.common.MessageFormatter import kafka.utils._ import kafka.utils.Implicits._ -import org.apache.kafka.clients.consumer.{ConsumerConfig, ConsumerRecord, KafkaConsumer} -import org.apache.kafka.common.errors.{AuthenticationException, WakeupException} +import org.apache.kafka.clients.consumer.{Consumer, ConsumerConfig, ConsumerRecord, KafkaConsumer} +import org.apache.kafka.common.TopicPartition +import org.apache.kafka.common.errors.{AuthenticationException, TimeoutException, WakeupException} import org.apache.kafka.common.record.TimestampType +import org.apache.kafka.common.requests.ListOffsetRequest import org.apache.kafka.common.serialization.{ByteArrayDeserializer, Deserializer} import org.apache.kafka.common.utils.Utils @@ -63,57 +62,30 @@ object ConsoleConsumer extends Logging { } def run(conf: ConsumerConfig) { + val timeoutMs = if (conf.timeoutMs >= 0) conf.timeoutMs else Long.MaxValue + val consumer = new KafkaConsumer(consumerProps(conf), new ByteArrayDeserializer, new ByteArrayDeserializer) + val consumerWrapper = + if (conf.partitionArg.isDefined) + new ConsumerWrapper(Option(conf.topicArg), conf.partitionArg, Option(conf.offsetArg), None, consumer, timeoutMs) + else + new ConsumerWrapper(Option(conf.topicArg), None, None, Option(conf.whitelistArg), consumer, timeoutMs) - val consumer = - if (conf.useOldConsumer) { - checkZk(conf) - val props = getOldConsumerProps(conf) - checkAndMaybeDeleteOldPath(conf, props) - new OldConsumer(conf.filterSpec, props) - } else { - val timeoutMs = if (conf.timeoutMs >= 0) conf.timeoutMs else Long.MaxValue - val consumer = new KafkaConsumer(getNewConsumerProps(conf), new ByteArrayDeserializer, new ByteArrayDeserializer) - if (conf.partitionArg.isDefined) - new NewShinyConsumer(Option(conf.topicArg), conf.partitionArg, Option(conf.offsetArg), None, consumer, timeoutMs) - else - new NewShinyConsumer(Option(conf.topicArg), None, None, Option(conf.whitelistArg), consumer, timeoutMs) - } - - addShutdownHook(consumer, conf) + addShutdownHook(consumerWrapper, conf) - try { - process(conf.maxMessages, conf.formatter, consumer, System.out, conf.skipMessageOnError) - } finally { - consumer.cleanup() + try process(conf.maxMessages, conf.formatter, consumerWrapper, System.out, conf.skipMessageOnError) + finally { + consumerWrapper.cleanup() conf.formatter.close() 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.useOldConsumer && !conf.groupIdPassed) - ZkUtils.maybeDeletePath(conf.options.valueOf(conf.zkConnectOpt), "/consumers/" + conf.consumerProps.get("group.id")) - shutdownLatch.countDown() } } - def checkZk(config: ConsumerConfig) { - if (!checkZkPathExists(config.options.valueOf(config.zkConnectOpt), "/brokers/ids")) { - System.err.println("No brokers found in ZK.") - Exit.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") - Exit.exit(1) - } - } - - def addShutdownHook(consumer: BaseConsumer, conf: ConsumerConfig) { + def addShutdownHook(consumer: ConsumerWrapper, conf: ConsumerConfig) { Runtime.getRuntime.addShutdownHook(new Thread() { override def run() { - consumer.stop() + consumer.wakeup() shutdownLatch.await() @@ -124,15 +96,12 @@ object ConsoleConsumer extends Logging { }) } - def process(maxMessages: Integer, formatter: MessageFormatter, consumer: BaseConsumer, output: PrintStream, skipMessageOnError: Boolean) { + def process(maxMessages: Integer, formatter: MessageFormatter, consumer: ConsumerWrapper, output: PrintStream, + skipMessageOnError: Boolean) { while (messageCount < maxMessages || maxMessages == -1) { - val msg: BaseConsumerRecord = try { + val msg: ConsumerRecord[Array[Byte], Array[Byte]] = try { consumer.receive() } catch { - case _: StreamEndException => - trace("Caught StreamEndException because consumer is shutdown, ignore and terminate.") - // Consumer is already closed - return case _: WakeupException => trace("Caught WakeupException because consumer is shutdown, ignore and terminate.") // Consumer will be closed @@ -175,35 +144,7 @@ object ConsoleConsumer extends Logging { gotError } - def getOldConsumerProps(config: ConsumerConfig): Properties = { - val props = new Properties - - props ++= config.consumerProps - props ++= config.extraConsumerProps - setAutoOffsetResetValue(config, props) - props.put("zookeeper.connect", config.zkConnectionStr) - - if (config.timeoutMs >= 0) - props.put("consumer.timeout.ms", config.timeoutMs.toString) - - props - } - - def checkAndMaybeDeleteOldPath(config: ConsumerConfig, props: Properties) = { - val consumerGroupBasePath = "/consumers/" + props.getProperty("group.id") - if (config.options.has(config.deleteConsumerOffsetsOpt)) { - ZkUtils.maybeDeletePath(config.options.valueOf(config.zkConnectOpt), consumerGroupBasePath) - } else { - val resetToBeginning = OffsetRequest.SmallestTimeString == props.getProperty(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG) - if (resetToBeginning && checkZkPathExists(config.options.valueOf(config.zkConnectOpt), consumerGroupBasePath + "/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") - Exit.exit(1) - } - } - } - - private[tools] def getNewConsumerProps(config: ConsumerConfig): Properties = { + private[tools] def consumerProps(config: ConsumerConfig): Properties = { val props = new Properties props ++= config.consumerProps props ++= config.extraConsumerProps @@ -214,8 +155,8 @@ object ConsoleConsumer extends Logging { } /** - * Used by both getNewConsumerProps and getOldConsumerProps to retrieve the correct value for the - * consumer parameter 'auto.offset.reset'. + * Used by consumerProps to retrieve the correct value for the consumer parameter 'auto.offset.reset'. + * * Order of priority is: * 1. Explicitly set parameter via --consumer.property command line parameter * 2. Explicit --from-beginning given -> 'earliest' @@ -225,10 +166,7 @@ object ConsoleConsumer extends Logging { * are conflicting. */ def setAutoOffsetResetValue(config: ConsumerConfig, props: Properties) { - val (earliestConfigValue, latestConfigValue) = if (config.useOldConsumer) - (OffsetRequest.SmallestTimeString, OffsetRequest.LargestTimeString) - else - ("earliest", "latest") + val (earliestConfigValue, latestConfigValue) = ("earliest", "latest") if (props.containsKey(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG)) { // auto.offset.reset parameter was specified on the command line @@ -259,10 +197,6 @@ object ConsoleConsumer extends Logging { .withRequiredArg .describedAs("whitelist") .ofType(classOf[String]) - val blacklistOpt = parser.accepts("blacklist", "Blacklist of topics to exclude from consumption.") - .withRequiredArg - .describedAs("blacklist") - .ofType(classOf[String]) val partitionIdOpt = parser.accepts("partition", "The partition to consume from. Consumption " + "starts from the end of the partition unless '--offset' is specified.") .withRequiredArg @@ -273,11 +207,6 @@ object ConsoleConsumer extends Logging { .describedAs("consume offset") .ofType(classOf[String]) .defaultsTo("latest") - val zkConnectOpt = parser.accepts("zookeeper", "REQUIRED (only when using old consumer): 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 consumerPropertyOpt = parser.accepts("consumer-property", "A mechanism to pass user-defined properties in the form key=value to the consumer.") .withRequiredArg .describedAs("consumer_prop") @@ -305,7 +234,6 @@ object ConsoleConsumer extends Logging { .withRequiredArg .describedAs("prop") .ofType(classOf[String]) - val deleteConsumerOffsetsOpt = parser.accepts("delete-consumer-offsets", "If specified, the consumer path in zookeeper is deleted when starting up") val resetBeginningOpt = parser.accepts("from-beginning", "If the consumer does not already have an established offset to consume from, " + "start with the earliest message present in the log rather than the latest message.") val maxMessagesOpt = parser.accepts("max-messages", "The maximum number of messages to consume before exiting. If not set, consumption is continual.") @@ -318,13 +246,7 @@ object ConsoleConsumer extends Logging { .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.") - 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 output here") - .withRequiredArg - .describedAs("metrics directory") - .ofType(classOf[java.lang.String]) - val bootstrapServerOpt = parser.accepts("bootstrap-server", "REQUIRED (unless old consumer is used): The server to connect to.") + val bootstrapServerOpt = parser.accepts("bootstrap-server", "REQUIRED: The server(s) to connect to.") .withRequiredArg .describedAs("server to connect to") .ofType(classOf[String]) @@ -337,10 +259,10 @@ object ConsoleConsumer extends Logging { .describedAs("deserializer for values") .ofType(classOf[String]) val enableSystestEventsLoggingOpt = parser.accepts("enable-systest-events", - "Log lifecycle events of the consumer in addition to logging consumed " + - "messages. (This is specific for system tests.)") + "Log lifecycle events of the consumer in addition to logging consumed " + + "messages. (This is specific for system tests.)") val isolationLevelOpt = parser.accepts("isolation-level", - "Set to read_committed in order to filter out transactional messages which are not committed. Set to read_uncommitted" + + "Set to read_committed in order to filter out transactional messages which are not committed. Set to read_uncommitted" + "to read all messages.") .withRequiredArg() .ofType(classOf[String]) @@ -356,11 +278,9 @@ object ConsoleConsumer extends Logging { var groupIdPassed = true val options: OptionSet = tryParse(parser, args) - val useOldConsumer = options.has(zkConnectOpt) val enableSystestEventsLogging = options.has(enableSystestEventsLoggingOpt) - // If using old consumer, exactly one of whitelist/blacklist/topic is required. - // If using new consumer, topic must be specified. + // topic must be specified. var topicArg: String = null var whitelistArg: String = null var filterSpec: TopicFilter = null @@ -369,7 +289,6 @@ object ConsoleConsumer extends Logging { Utils.loadProps(options.valueOf(consumerConfigOpt)) else new Properties() - val zkConnectionStr = options.valueOf(zkConnectOpt) val fromBeginning = options.has(resetBeginningOpt) val partitionArg = if (options.has(partitionIdOpt)) Some(options.valueOf(partitionIdOpt).intValue) else None val skipMessageOnError = options.has(skipMessageOnErrorOpt) @@ -392,26 +311,11 @@ object ConsoleConsumer extends Logging { formatter.init(formatterArgs) - if (useOldConsumer) { - if (options.has(bootstrapServerOpt)) - CommandLineUtils.printUsageAndDie(parser, s"Option $bootstrapServerOpt is not valid with $zkConnectOpt.") - val topicOrFilterOpt = List(topicIdOpt, whitelistOpt, blacklistOpt).filter(options.has) - if (topicOrFilterOpt.size != 1) - CommandLineUtils.printUsageAndDie(parser, "Exactly one of whitelist/blacklist/topic is required.") - topicArg = options.valueOf(topicOrFilterOpt.head) - filterSpec = if (options.has(blacklistOpt)) new Blacklist(topicArg) else new Whitelist(topicArg) - Console.err.println("Using the ConsoleConsumer with old consumer is deprecated and will be removed " + - s"in a future major release. Consider using the new consumer by passing $bootstrapServerOpt instead of ${zkConnectOpt}.") - } else { - val topicOrFilterOpt = List(topicIdOpt, whitelistOpt).filter(options.has) - if (topicOrFilterOpt.size != 1) - CommandLineUtils.printUsageAndDie(parser, "Exactly one of whitelist/topic is required.") - topicArg = options.valueOf(topicIdOpt) - whitelistArg = options.valueOf(whitelistOpt) - } - - if (useOldConsumer && (partitionArg.isDefined || options.has(offsetOpt))) - CommandLineUtils.printUsageAndDie(parser, "Partition-offset based consumption is supported in the new consumer only.") + val topicOrFilterOpt = List(topicIdOpt, whitelistOpt).filter(options.has) + if (topicOrFilterOpt.size != 1) + CommandLineUtils.printUsageAndDie(parser, "Exactly one of whitelist/topic is required.") + topicArg = options.valueOf(topicIdOpt) + whitelistArg = options.valueOf(whitelistOpt) if (partitionArg.isDefined) { if (!options.has(topicIdOpt)) @@ -428,49 +332,35 @@ object ConsoleConsumer extends Logging { val offsetArg = if (options.has(offsetOpt)) { options.valueOf(offsetOpt).toLowerCase(Locale.ROOT) match { - case "earliest" => OffsetRequest.EarliestTime - case "latest" => OffsetRequest.LatestTime + case "earliest" => ListOffsetRequest.EARLIEST_TIMESTAMP + case "latest" => ListOffsetRequest.LATEST_TIMESTAMP case offsetString => - val offset = - try offsetString.toLong - catch { - case _: NumberFormatException => invalidOffset(offsetString) - } - if (offset < 0) invalidOffset(offsetString) - offset + try { + val offset = offsetString.toLong + if (offset < 0) + invalidOffset(offsetString) + offset + } catch { + case _: NumberFormatException => invalidOffset(offsetString) + } } } - else if (fromBeginning) OffsetRequest.EarliestTime - else OffsetRequest.LatestTime - - if (!useOldConsumer) { - CommandLineUtils.checkRequiredArgs(parser, options, bootstrapServerOpt) - } + else if (fromBeginning) ListOffsetRequest.EARLIEST_TIMESTAMP + else ListOffsetRequest.LATEST_TIMESTAMP - if (options.has(csvMetricsReporterEnabledOpt)) { - val csvReporterProps = new Properties() - csvReporterProps.put("kafka.metrics.polling.interval.secs", "5") - csvReporterProps.put("kafka.metrics.reporters", "kafka.metrics.KafkaCSVMetricsReporter") - if (options.has(metricsDirectoryOpt)) - csvReporterProps.put("kafka.csv.metrics.dir", options.valueOf(metricsDirectoryOpt)) - else - csvReporterProps.put("kafka.csv.metrics.dir", "kafka_metrics") - csvReporterProps.put("kafka.csv.metrics.reporter.enabled", "true") - val verifiableProps = new VerifiableProperties(csvReporterProps) - KafkaMetricsReporter.startReporters(verifiableProps) - } + CommandLineUtils.checkRequiredArgs(parser, options, bootstrapServerOpt) // if the group id is provided in more than place (through different means) all values must be the same val groupIdsProvided = Set( - Option(options.valueOf(groupIdOpt)), // via --group - Option(consumerProps.get(ConsumerConfig.GROUP_ID_CONFIG)), // via --consumer-property - Option(extraConsumerProps.get(ConsumerConfig.GROUP_ID_CONFIG)) // via --cosumer.config - ).flatten + Option(options.valueOf(groupIdOpt)), // via --group + Option(consumerProps.get(ConsumerConfig.GROUP_ID_CONFIG)), // via --consumer-property + Option(extraConsumerProps.get(ConsumerConfig.GROUP_ID_CONFIG)) // via --cosumer.config + ).flatten if (groupIdsProvided.size > 1) { CommandLineUtils.printUsageAndDie(parser, "The group ids provided in different places (directly using '--group', " - + "via '--consumer-property', or via '--consumer.config') do not match. " - + s"Detected group ids: ${groupIdsProvided.mkString("'", "', '", "'")}") + + "via '--consumer-property', or via '--consumer.config') do not match. " + + s"Detected group ids: ${groupIdsProvided.mkString("'", "', '", "'")}") } groupIdsProvided.headOption match { @@ -491,12 +381,72 @@ object ConsoleConsumer extends Logging { } } - def checkZkPathExists(zkUrl: String, path: String): Boolean = { - try { - val zk = ZkUtils.createZkClient(zkUrl, 30 * 1000, 30 * 1000) - zk.exists(path) - } catch { - case _: Throwable => false + private[tools] class ConsumerWrapper(topic: Option[String], partitionId: Option[Int], offset: Option[Long], whitelist: Option[String], + consumer: Consumer[Array[Byte], Array[Byte]], val timeoutMs: Long = Long.MaxValue) { + consumerInit() + var recordIter = consumer.poll(0).iterator + + def consumerInit() { + (topic, partitionId, offset, whitelist) match { + case (Some(topic), Some(partitionId), Some(offset), None) => + seek(topic, partitionId, offset) + case (Some(topic), Some(partitionId), None, None) => + // default to latest if no offset is provided + seek(topic, partitionId, ListOffsetRequest.LATEST_TIMESTAMP) + case (Some(topic), None, None, None) => + consumer.subscribe(Collections.singletonList(topic)) + case (None, None, None, Some(whitelist)) => + consumer.subscribe(Pattern.compile(whitelist)) + case _ => + throw new IllegalArgumentException("An invalid combination of arguments is provided. " + + "Exactly one of 'topic' or 'whitelist' must be provided. " + + "If 'topic' is provided, an optional 'partition' may also be provided. " + + "If 'partition' is provided, an optional 'offset' may also be provided, otherwise, consumption starts from the end of the partition.") + } + } + + def seek(topic: String, partitionId: Int, offset: Long) { + val topicPartition = new TopicPartition(topic, partitionId) + consumer.assign(Collections.singletonList(topicPartition)) + offset match { + case ListOffsetRequest.EARLIEST_TIMESTAMP => consumer.seekToBeginning(Collections.singletonList(topicPartition)) + case ListOffsetRequest.LATEST_TIMESTAMP => consumer.seekToEnd(Collections.singletonList(topicPartition)) + case _ => consumer.seek(topicPartition, offset) + } + } + + def resetUnconsumedOffsets() { + val smallestUnconsumedOffsets = collection.mutable.Map[TopicPartition, Long]() + while (recordIter.hasNext) { + val record = recordIter.next() + val tp = new TopicPartition(record.topic, record.partition) + // avoid auto-committing offsets which haven't been consumed + smallestUnconsumedOffsets.getOrElseUpdate(tp, record.offset) + } + smallestUnconsumedOffsets.foreach { case (tp, offset) => consumer.seek(tp, offset) } + } + + def receive(): ConsumerRecord[Array[Byte], Array[Byte]] = { + if (!recordIter.hasNext) { + recordIter = consumer.poll(timeoutMs).iterator + if (!recordIter.hasNext) + throw new TimeoutException() + } + + recordIter.next + } + + def wakeup(): Unit = { + this.consumer.wakeup() + } + + def cleanup() { + resetUnconsumedOffsets() + this.consumer.close() + } + + def commitSync() { + this.consumer.commitSync() } } } @@ -613,12 +563,6 @@ class ChecksumMessageFormatter extends MessageFormatter { } def writeTo(consumerRecord: ConsumerRecord[Array[Byte], Array[Byte]], output: PrintStream) { - import consumerRecord._ - val chksum = - if (timestampType != TimestampType.NO_TIMESTAMP_TYPE) - new Message(value, key, timestamp, timestampType, NoCompressionCodec, 0, -1, Message.MagicValue_V1).checksum - else - new Message(value, key, Message.NoTimestamp, Message.MagicValue_V0).checksum - output.println(topicStr + "checksum:" + chksum) + output.println(topicStr + "checksum:" + consumerRecord.checksum) } } diff --git a/core/src/main/scala/kafka/tools/ConsoleProducer.scala b/core/src/main/scala/kafka/tools/ConsoleProducer.scala index e5b72a3fa1979..8d8c42d36cf07 100644 --- a/core/src/main/scala/kafka/tools/ConsoleProducer.scala +++ b/core/src/main/scala/kafka/tools/ConsoleProducer.scala @@ -28,6 +28,7 @@ import java.nio.charset.StandardCharsets import joptsimple._ import org.apache.kafka.clients.producer.internals.ErrorLoggingCallback import org.apache.kafka.clients.producer.{KafkaProducer, ProducerConfig, ProducerRecord} +import org.apache.kafka.common.KafkaException import org.apache.kafka.common.utils.Utils import scala.collection.JavaConverters._ @@ -228,7 +229,6 @@ object ConsoleProducer { val socketBuffer = options.valueOf(socketBufferSizeOpt) val cmdLineProps = CommandLineUtils.parseKeyValueArgs(options.valuesOf(propertyOpt).asScala) val extraProducerProps = CommandLineUtils.parseKeyValueArgs(options.valuesOf(producerPropertyOpt).asScala) - /* new producer related configs */ val maxMemoryBytes = options.valueOf(maxMemoryBytesOpt) val maxPartitionMemoryBytes = options.valueOf(maxPartitionMemoryBytesOpt) val metadataExpiryMs = options.valueOf(metadataExpiryMsOpt) diff --git a/core/src/main/scala/kafka/tools/ConsumerPerformance.scala b/core/src/main/scala/kafka/tools/ConsumerPerformance.scala index f4221fe294641..5af55a8d7f17b 100644 --- a/core/src/main/scala/kafka/tools/ConsumerPerformance.scala +++ b/core/src/main/scala/kafka/tools/ConsumerPerformance.scala @@ -21,7 +21,6 @@ import java.util import scala.collection.JavaConverters._ import java.util.concurrent.atomic.AtomicLong -import java.nio.channels.ClosedByInterruptException import org.apache.kafka.clients.consumer.{ConsumerRebalanceListener, KafkaConsumer} import org.apache.kafka.common.serialization.ByteArrayDeserializer @@ -30,12 +29,7 @@ import org.apache.kafka.common.{Metric, MetricName, TopicPartition} import kafka.utils.{CommandLineUtils, ToolsUtils} import java.util.{Collections, Properties, Random} -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 import com.typesafe.scalalogging.LazyLogging @@ -52,70 +46,39 @@ object ConsumerPerformance extends LazyLogging { logger.info("Starting consumer...") val totalMessagesRead = new AtomicLong(0) val totalBytesRead = new AtomicLong(0) - val consumerTimeout = new AtomicBoolean(false) var metrics: mutable.Map[MetricName, _ <: Metric] = null val joinGroupTimeInMs = new AtomicLong(0) - if (!config.hideHeader) { - printHeader(config.showDetailedStats, config.useOldConsumer) - } + if (!config.hideHeader) + printHeader(config.showDetailedStats) var startMs, endMs = 0L - if (!config.useOldConsumer) { - val consumer = new KafkaConsumer[Array[Byte], Array[Byte]](config.props) - consumer.subscribe(Collections.singletonList(config.topic)) - startMs = System.currentTimeMillis - consume(consumer, List(config.topic), config.numMessages, config.recordFetchTimeoutMs, config, totalMessagesRead, totalBytesRead, joinGroupTimeInMs, startMs) - endMs = System.currentTimeMillis - - if (config.printMetrics) { - metrics = consumer.metrics().asScala - } - 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 (streamList <- topicMessageStreams.values) - for (i <- 0 until streamList.length) - threadList ::= new ConsumerPerfThread(i, "kafka-zk-consumer-" + i, streamList(i), config, totalMessagesRead, totalBytesRead, consumerTimeout) - - logger.info("Sleeping for 1 second.") - Thread.sleep(1000) - logger.info("starting threads") - startMs = System.currentTimeMillis - for (thread <- threadList) - thread.start() - for (thread <- threadList) - thread.join() - endMs = - if (consumerTimeout.get()) System.currentTimeMillis - consumerConfig.consumerTimeoutMs - else System.currentTimeMillis - consumerConnector.shutdown() + val consumer = new KafkaConsumer[Array[Byte], Array[Byte]](config.props) + consumer.subscribe(Collections.singletonList(config.topic)) + startMs = System.currentTimeMillis + consume(consumer, List(config.topic), config.numMessages, config.recordFetchTimeoutMs, config, totalMessagesRead, totalBytesRead, joinGroupTimeInMs, startMs) + endMs = System.currentTimeMillis + + if (config.printMetrics) { + metrics = consumer.metrics.asScala } + consumer.close() val elapsedSecs = (endMs - startMs) / 1000.0 val fetchTimeInMs = (endMs - startMs) - joinGroupTimeInMs.get if (!config.showDetailedStats) { val totalMBRead = (totalBytesRead.get * 1.0) / (1024 * 1024) - print("%s, %s, %.4f, %.4f, %d, %.4f".format( + println("%s, %s, %.4f, %.4f, %d, %.4f, %d, %d, %.4f, %.4f".format( config.dateFormat.format(startMs), config.dateFormat.format(endMs), totalMBRead, totalMBRead / elapsedSecs, totalMessagesRead.get, - totalMessagesRead.get / elapsedSecs + totalMessagesRead.get / elapsedSecs, + joinGroupTimeInMs.get, + fetchTimeInMs, + totalMBRead / (fetchTimeInMs / 1000.0), + totalMessagesRead.get / (fetchTimeInMs / 1000.0) )) - if (!config.useOldConsumer) { - print(", %d, %d, %.4f, %.4f".format( - joinGroupTimeInMs.get, - fetchTimeInMs, - totalMBRead / (fetchTimeInMs / 1000.0), - totalMessagesRead.get / (fetchTimeInMs / 1000.0) - )) - } - println() } if (metrics != null) { @@ -124,13 +87,12 @@ object ConsumerPerformance extends LazyLogging { } - private[tools] def printHeader(showDetailedStats: Boolean, useOldConsumer: Boolean): Unit = { - val newFieldsInHeader = if (!useOldConsumer) ", rebalance.time.ms, fetch.time.ms, fetch.MB.sec, fetch.nMsg.sec" else "" - if (!showDetailedStats) { - println("start.time, end.time, data.consumed.in.MB, MB.sec, data.consumed.in.nMsg, nMsg.sec" + newFieldsInHeader) - } else { - println("time, threadId, data.consumed.in.MB, MB.sec, data.consumed.in.nMsg, nMsg.sec" + newFieldsInHeader) - } + private[tools] def printHeader(showDetailedStats: Boolean): Unit = { + val newFieldsInHeader = ", rebalance.time.ms, fetch.time.ms, fetch.MB.sec, fetch.nMsg.sec" + if (!showDetailedStats) + println("start.time, end.time, data.consumed.in.MB, MB.sec, data.consumed.in.nMsg, nMsg.sec" + newFieldsInHeader) + else + println("time, threadId, data.consumed.in.MB, MB.sec, data.consumed.in.nMsg, nMsg.sec" + newFieldsInHeader) } def consume(consumer: KafkaConsumer[Array[Byte], Array[Byte]], @@ -178,7 +140,7 @@ object ConsumerPerformance extends LazyLogging { if (currentTimeMillis - lastReportTime >= config.reportingInterval) { if (config.showDetailedStats) - printNewConsumerProgress(0, bytesRead, lastBytesRead, messagesRead, lastMessagesRead, + printConsumerProgress(0, bytesRead, lastBytesRead, messagesRead, lastMessagesRead, lastReportTime, currentTimeMillis, config.dateFormat, joinTimeMsInSingleRound) joinTimeMsInSingleRound = 0L lastReportTime = currentTimeMillis @@ -195,19 +157,7 @@ object ConsumerPerformance extends LazyLogging { totalBytesRead.set(bytesRead) } - def printOldConsumerProgress(id: Int, - bytesRead: Long, - lastBytesRead: Long, - messagesRead: Long, - lastMessagesRead: Long, - startMs: Long, - endMs: Long, - dateFormat: SimpleDateFormat): Unit = { - printBasicProgress(id, bytesRead, lastBytesRead, messagesRead, lastMessagesRead, startMs, endMs, dateFormat) - println() - } - - def printNewConsumerProgress(id: Int, + def printConsumerProgress(id: Int, bytesRead: Long, lastBytesRead: Long, messagesRead: Long, @@ -256,12 +206,7 @@ object ConsumerPerformance extends LazyLogging { } class ConsumerPerfConfig(args: Array[String]) extends PerfConfig(args) { - val zkConnectOpt = parser.accepts("zookeeper", "REQUIRED (only when using old consumer): 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", "REQUIRED (unless old consumer is used): A broker list to use for connecting if using the new consumer.") + val bootstrapServersOpt = parser.accepts("broker-list", "REQUIRED: The server(s) to connect to.") .withRequiredArg() .describedAs("host") .ofType(classOf[String]) @@ -300,7 +245,7 @@ object ConsumerPerformance extends LazyLogging { .withRequiredArg .describedAs("config file") .ofType(classOf[String]) - val printMetricsOpt = parser.accepts("print-metrics", "Print out the metrics. This only applies to new consumer.") + val printMetricsOpt = parser.accepts("print-metrics", "Print out the metrics.") val showDetailedStatsOpt = parser.accepts("show-detailed-stats", "If set, stats are reported for each reporting " + "interval as configured by reporting-interval") val recordFetchTimeoutOpt = parser.accepts("timeout", "The maximum allowed time in milliseconds between returned records.") @@ -311,40 +256,25 @@ object ConsumerPerformance extends LazyLogging { val options = parser.parse(args: _*) - CommandLineUtils.checkRequiredArgs(parser, options, topicOpt, numMessagesOpt) + CommandLineUtils.checkRequiredArgs(parser, options, topicOpt, numMessagesOpt, bootstrapServersOpt) - val useOldConsumer = options.has(zkConnectOpt) val printMetrics = options.has(printMetricsOpt) val props = if (options.has(consumerConfigOpt)) Utils.loadProps(options.valueOf(consumerConfigOpt)) else new Properties - if (!useOldConsumer) { - CommandLineUtils.checkRequiredArgs(parser, options, bootstrapServersOpt) - - 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 { - if (options.has(bootstrapServersOpt)) - CommandLineUtils.printUsageAndDie(parser, s"Option $bootstrapServersOpt is not valid with $zkConnectOpt.") - - 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) - } + + 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") + val numThreads = options.valueOf(numThreadsOpt).intValue val topic = options.valueOf(topicOpt) val numMessages = options.valueOf(numMessagesOpt).longValue @@ -356,53 +286,4 @@ object ConsumerPerformance extends LazyLogging { val hideHeader = options.has(hideHeaderOpt) val recordFetchTimeoutMs = options.valueOf(recordFetchTimeoutOpt).longValue() } - - class ConsumerPerfThread(threadId: Int, - name: String, - stream: KafkaStream[Array[Byte], Array[Byte]], - config: ConsumerPerfConfig, - totalMessagesRead: AtomicLong, - totalBytesRead: AtomicLong, - consumerTimeout: AtomicBoolean) - extends Thread(name) { - - override def run() { - var bytesRead = 0L - var messagesRead = 0L - val startMs = System.currentTimeMillis - var lastReportTime: Long = startMs - var lastBytesRead = 0L - var lastMessagesRead = 0L - - try { - val iter = stream.iterator - while (iter.hasNext && messagesRead < config.numMessages) { - val messageAndMetadata = iter.next() - messagesRead += 1 - bytesRead += messageAndMetadata.message.length - val currentTimeMillis = System.currentTimeMillis - - if (currentTimeMillis - lastReportTime >= config.reportingInterval) { - if (config.showDetailedStats) - printOldConsumerProgress(threadId, bytesRead, lastBytesRead, messagesRead, lastMessagesRead, lastReportTime, currentTimeMillis, config.dateFormat) - lastReportTime = currentTimeMillis - lastMessagesRead = messagesRead - lastBytesRead = bytesRead - } - } - } catch { - case _: InterruptedException => - case _: ClosedByInterruptException => - case _: ConsumerTimeoutException => - consumerTimeout.set(true) - case e: Throwable => e.printStackTrace() - } - totalMessagesRead.addAndGet(messagesRead) - totalBytesRead.addAndGet(bytesRead) - if (config.showDetailedStats) - printOldConsumerProgress(threadId, bytesRead, lastBytesRead, messagesRead, lastMessagesRead, startMs, System.currentTimeMillis, config.dateFormat) - - } - - } } diff --git a/core/src/main/scala/kafka/tools/DumpLogSegments.scala b/core/src/main/scala/kafka/tools/DumpLogSegments.scala index 2aa7ad3495a6d..0f588e36109a6 100755 --- a/core/src/main/scala/kafka/tools/DumpLogSegments.scala +++ b/core/src/main/scala/kafka/tools/DumpLogSegments.scala @@ -189,7 +189,7 @@ object DumpLogSegments { for(i <- 0 until index.entries) { val entry = index.entry(i) - val slice = fileRecords.read(entry.position, maxMessageSize) + val slice = fileRecords.slice(entry.position, maxMessageSize) val firstRecord = slice.records.iterator.next() if (firstRecord.offset != entry.offset + index.baseOffset) { var misMatchesSeq = misMatchesForIndexFilesMap.getOrElse(file.getAbsolutePath, List[(Long, Long)]()) @@ -227,7 +227,7 @@ object DumpLogSegments { for(i <- 0 until timeIndex.entries) { val entry = timeIndex.entry(i) val position = index.lookup(entry.offset + timeIndex.baseOffset).position - val partialFileRecords = fileRecords.read(position, Int.MaxValue) + val partialFileRecords = fileRecords.slice(position, Int.MaxValue) val batches = partialFileRecords.batches.asScala var maxTimestamp = RecordBatch.NO_TIMESTAMP // We first find the message by offset then check if the timestamp is correct. @@ -424,7 +424,8 @@ object DumpLogSegments { print("baseOffset: " + batch.baseOffset + " lastOffset: " + batch.lastOffset + " count: " + batch.countOrNull + " baseSequence: " + batch.baseSequence + " lastSequence: " + batch.lastSequence + " producerId: " + batch.producerId + " producerEpoch: " + batch.producerEpoch + - " partitionLeaderEpoch: " + batch.partitionLeaderEpoch + " isTransactional: " + batch.isTransactional) + " partitionLeaderEpoch: " + batch.partitionLeaderEpoch + " isTransactional: " + batch.isTransactional + + " isControl: " + batch.isControlBatch) else print("offset: " + batch.lastOffset) diff --git a/core/src/main/scala/kafka/tools/ExportZkOffsets.scala b/core/src/main/scala/kafka/tools/ExportZkOffsets.scala deleted file mode 100644 index d8ce9b068acb7..0000000000000 --- a/core/src/main/scala/kafka/tools/ExportZkOffsets.scala +++ /dev/null @@ -1,132 +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 java.io.{FileOutputStream, OutputStreamWriter} -import java.nio.charset.StandardCharsets - -import joptsimple._ -import kafka.utils.{CommandLineUtils, Exit, Logging, ZKGroupTopicDirs, ZkUtils} -import org.apache.kafka.common.security.JaasUtils - -import scala.collection.JavaConverters._ - - -/** - * A utility that retrieves the offset of broker partitions in ZK and - * prints to an output file in the following format: - * - * /consumers/group1/offsets/topic1/1-0:286894308 - * /consumers/group1/offsets/topic1/2-0:284803985 - * - * This utility expects 3 arguments: - * 1. Zk host:port string - * 2. group name (all groups implied if omitted) - * 3. output filename - * - * To print debug message, add the following line to log4j.properties: - * log4j.logger.kafka.tools.ExportZkOffsets$=DEBUG - * (for eclipse debugging, copy log4j.properties to the binary directory in "core" such as core/bin) - */ -@deprecated("This class has been deprecated and will be removed in a future release.", "0.11.0.0") -object ExportZkOffsets extends Logging { - - def main(args: Array[String]) { - val parser = new OptionParser(false) - warn("WARNING: ExportZkOffsets is deprecated and will be dropped in a future release following 0.11.0.0.") - - val zkConnectOpt = parser.accepts("zkconnect", "ZooKeeper connect string.") - .withRequiredArg() - .defaultsTo("localhost:2181") - .ofType(classOf[String]) - val groupOpt = parser.accepts("group", "Consumer group.") - .withRequiredArg() - .ofType(classOf[String]) - val outFileOpt = parser.accepts("output-file", "Output file") - .withRequiredArg() - .ofType(classOf[String]) - parser.accepts("help", "Print this message.") - - if(args.length == 0) - CommandLineUtils.printUsageAndDie(parser, "Export consumer offsets to an output file.") - - val options = parser.parse(args : _*) - - if (options.has("help")) { - parser.printHelpOn(System.out) - Exit.exit(0) - } - - CommandLineUtils.checkRequiredArgs(parser, options, zkConnectOpt, outFileOpt) - - val zkConnect = options.valueOf(zkConnectOpt) - val groups = options.valuesOf(groupOpt) - val outfile = options.valueOf(outFileOpt) - - var zkUtils : ZkUtils = null - val fileWriter : OutputStreamWriter = - new OutputStreamWriter(new FileOutputStream(outfile), StandardCharsets.UTF_8) - - try { - zkUtils = ZkUtils(zkConnect, - 30000, - 30000, - JaasUtils.isZkSecurityEnabled()) - - var consumerGroups: Seq[String] = null - - if (groups.size == 0) { - consumerGroups = zkUtils.getChildren(ZkUtils.ConsumersPath).toList - } - else { - consumerGroups = groups.asScala - } - - for (consumerGrp <- consumerGroups) { - val topicsList = getTopicsList(zkUtils, consumerGrp) - - for (topic <- topicsList) { - val bidPidList = getBrokeridPartition(zkUtils, consumerGrp, topic) - - for (bidPid <- bidPidList) { - val zkGrpTpDir = new ZKGroupTopicDirs(consumerGrp,topic) - val offsetPath = zkGrpTpDir.consumerOffsetDir + "/" + bidPid - zkUtils.readDataMaybeNull(offsetPath)._1 match { - case Some(offsetVal) => - fileWriter.write(offsetPath + ":" + offsetVal + "\n") - debug(offsetPath + " => " + offsetVal) - case None => - error("Could not retrieve offset value from " + offsetPath) - } - } - } - } - } - finally { - fileWriter.flush() - fileWriter.close() - } - } - - private def getBrokeridPartition(zkUtils: ZkUtils, consumerGroup: String, topic: String): List[String] = - zkUtils.getChildrenParentMayNotExist("/consumers/%s/offsets/%s".format(consumerGroup, topic)).toList - - private def getTopicsList(zkUtils: ZkUtils, consumerGroup: String): List[String] = - zkUtils.getChildren("/consumers/%s/offsets".format(consumerGroup)).toList - -} diff --git a/core/src/main/scala/kafka/tools/GetOffsetShell.scala b/core/src/main/scala/kafka/tools/GetOffsetShell.scala index 4104dedb9e09a..eafddc66de428 100644 --- a/core/src/main/scala/kafka/tools/GetOffsetShell.scala +++ b/core/src/main/scala/kafka/tools/GetOffsetShell.scala @@ -18,13 +18,16 @@ */ package kafka.tools -import kafka.consumer._ +import java.util.Properties + import joptsimple._ -import kafka.api.{OffsetRequest, PartitionOffsetRequestInfo} -import kafka.common.TopicAndPartition -import kafka.client.ClientUtils import kafka.utils.{CommandLineUtils, Exit, ToolsUtils} +import org.apache.kafka.clients.consumer.{ConsumerConfig, KafkaConsumer} +import org.apache.kafka.common.{PartitionInfo, TopicPartition} +import org.apache.kafka.common.requests.ListOffsetRequest +import org.apache.kafka.common.serialization.ByteArrayDeserializer +import scala.collection.JavaConverters._ object GetOffsetShell { @@ -47,20 +50,20 @@ object GetOffsetShell { .withRequiredArg .describedAs("timestamp/-1(latest)/-2(earliest)") .ofType(classOf[java.lang.Long]) - .defaultsTo(-1) - val nOffsetsOpt = parser.accepts("offsets", "number of offsets returned") + .defaultsTo(-1L) + parser.accepts("offsets", "DEPRECATED AND IGNORED: number of offsets returned") .withRequiredArg .describedAs("count") .ofType(classOf[java.lang.Integer]) .defaultsTo(1) - val maxWaitMsOpt = parser.accepts("max-wait-ms", "The max amount of time each fetch request waits.") + parser.accepts("max-wait-ms", "DEPRECATED AND IGNORED: The max amount of time each fetch request waits.") .withRequiredArg .describedAs("ms") .ofType(classOf[java.lang.Integer]) .defaultsTo(1000) - - if(args.length == 0) - CommandLineUtils.printUsageAndDie(parser, "An interactive shell for getting consumer offsets.") + + if (args.length == 0) + CommandLineUtils.printUsageAndDie(parser, "An interactive shell for getting topic offsets.") val options = parser.parse(args : _*) @@ -69,41 +72,81 @@ object GetOffsetShell { val clientId = "GetOffsetShell" val brokerList = options.valueOf(brokerListOpt) ToolsUtils.validatePortOrDie(parser, brokerList) - val metadataTargetBrokers = ClientUtils.parseBrokerList(brokerList) val topic = options.valueOf(topicOpt) - val partitionList = options.valueOf(partitionOpt) - val time = options.valueOf(timeOpt).longValue - val nOffsets = options.valueOf(nOffsetsOpt).intValue - val maxWaitMs = options.valueOf(maxWaitMsOpt).intValue() - - val topicsMetadata = ClientUtils.fetchTopicMetadata(Set(topic), metadataTargetBrokers, clientId, maxWaitMs).topicsMetadata - if(topicsMetadata.size != 1 || !topicsMetadata.head.topic.equals(topic)) { - System.err.println(("Error: no valid topic metadata for topic: %s, " + " probably the topic does not exist, run ").format(topic) + - "kafka-list-topic.sh to verify") - Exit.exit(1) - } - val partitions = - if(partitionList == "") { - topicsMetadata.head.partitionsMetadata.map(_.partitionId) - } else { - partitionList.split(",").map(_.toInt).toSeq - } - partitions.foreach { partitionId => - val partitionMetadataOpt = topicsMetadata.head.partitionsMetadata.find(_.partitionId == partitionId) - partitionMetadataOpt match { - case Some(metadata) => - metadata.leader match { - case Some(leader) => - val consumer = new SimpleConsumer(leader.host, leader.port, 10000, 100000, clientId) - val topicAndPartition = TopicAndPartition(topic, partitionId) - val request = OffsetRequest(Map(topicAndPartition -> PartitionOffsetRequestInfo(time, nOffsets))) - val offsets = consumer.getOffsetsBefore(request).partitionErrorAndOffsets(topicAndPartition).offsets - - println("%s:%d:%s".format(topic, partitionId, offsets.mkString(","))) - case None => System.err.println("Error: partition %d does not have a leader. Skip getting offsets".format(partitionId)) + val partitionIdsRequested: Set[Int] = { + val partitionsString = options.valueOf(partitionOpt) + if (partitionsString.isEmpty) + Set.empty + else + partitionsString.split(",").map { partitionString => + try partitionString.toInt + catch { + case _: NumberFormatException => + System.err.println(s"--partitions expects a comma separated list of numeric partition ids, but received: $partitionsString") + Exit.exit(1) } - case None => System.err.println("Error: partition %d does not exist".format(partitionId)) + }.toSet + } + val listOffsetsTimestamp = options.valueOf(timeOpt).longValue + + val config = new Properties + config.setProperty(ConsumerConfig.BOOTSTRAP_SERVERS_CONFIG, brokerList) + config.setProperty(ConsumerConfig.CLIENT_ID_CONFIG, clientId) + val consumer = new KafkaConsumer(config, new ByteArrayDeserializer, new ByteArrayDeserializer) + + val partitionInfos = listPartitionInfos(consumer, topic, partitionIdsRequested) match { + case None => + System.err.println(s"Topic $topic does not exist") + Exit.exit(1) + case Some(p) if p.isEmpty => + if (partitionIdsRequested.isEmpty) + System.err.println(s"Topic $topic has 0 partitions") + else + System.err.println(s"Topic $topic does not have any of the requested partitions ${partitionIdsRequested.mkString(",")}") + Exit.exit(1) + case Some(p) => p + } + + if (partitionIdsRequested.nonEmpty) { + (partitionIdsRequested -- partitionInfos.map(_.partition)).foreach { partitionId => + System.err.println(s"Error: partition $partitionId does not exist") } } + + val topicPartitions = partitionInfos.sortBy(_.partition).flatMap { p => + if (p.leader == null) { + System.err.println(s"Error: partition ${p.partition} does not have a leader. Skip getting offsets") + None + } else + Some(new TopicPartition(p.topic, p.partition)) + } + + /* Note that the value of the map can be null */ + val partitionOffsets: collection.Map[TopicPartition, java.lang.Long] = listOffsetsTimestamp match { + case ListOffsetRequest.EARLIEST_TIMESTAMP => consumer.beginningOffsets(topicPartitions.asJava).asScala + case ListOffsetRequest.LATEST_TIMESTAMP => consumer.endOffsets(topicPartitions.asJava).asScala + case _ => + val timestampsToSearch = topicPartitions.map(tp => tp -> (listOffsetsTimestamp: java.lang.Long)).toMap.asJava + consumer.offsetsForTimes(timestampsToSearch).asScala.mapValues(x => if (x == null) null else x.offset) + } + + partitionOffsets.toSeq.sortBy { case (tp, _) => tp.partition }.foreach { case (tp, offset) => + println(s"$topic:${tp.partition}:${Option(offset).getOrElse("")}") + } + + } + + /** + * Return the partition infos for `topic`. If the topic does not exist, `None` is returned. + */ + private def listPartitionInfos(consumer: KafkaConsumer[_, _], topic: String, partitionIds: Set[Int]): Option[Seq[PartitionInfo]] = { + val partitionInfos = consumer.listTopics.asScala.filterKeys(_ == topic).values.flatMap(_.asScala).toBuffer + if (partitionInfos.isEmpty) + None + else if (partitionIds.isEmpty) + Some(partitionInfos) + else + Some(partitionInfos.filter(p => partitionIds.contains(p.partition))) } + } diff --git a/core/src/main/scala/kafka/tools/ImportZkOffsets.scala b/core/src/main/scala/kafka/tools/ImportZkOffsets.scala deleted file mode 100644 index c345f94dff4ba..0000000000000 --- a/core/src/main/scala/kafka/tools/ImportZkOffsets.scala +++ /dev/null @@ -1,110 +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 java.io.{BufferedReader, FileInputStream, InputStreamReader} -import java.nio.charset.StandardCharsets - -import joptsimple._ -import kafka.utils.{CommandLineUtils, Exit, Logging, ZkUtils} -import org.apache.kafka.common.security.JaasUtils - - -/** - * A utility that updates the offset of broker partitions in ZK. - * - * This utility expects 2 input files as arguments: - * 1. consumer properties file - * 2. a file contains partition offsets data such as: - * (This output data file can be obtained by running kafka.tools.ExportZkOffsets) - * - * /consumers/group1/offsets/topic1/3-0:285038193 - * /consumers/group1/offsets/topic1/1-0:286894308 - * - * To print debug message, add the following line to log4j.properties: - * log4j.logger.kafka.tools.ImportZkOffsets$=DEBUG - * (for eclipse debugging, copy log4j.properties to the binary directory in "core" such as core/bin) - */ -object ImportZkOffsets extends Logging { - - def main(args: Array[String]) { - val parser = new OptionParser(false) - - val zkConnectOpt = parser.accepts("zkconnect", "ZooKeeper connect string.") - .withRequiredArg() - .defaultsTo("localhost:2181") - .ofType(classOf[String]) - val inFileOpt = parser.accepts("input-file", "Input file") - .withRequiredArg() - .ofType(classOf[String]) - parser.accepts("help", "Print this message.") - - if(args.length == 0) - CommandLineUtils.printUsageAndDie(parser, "Import offsets to zookeeper from files.") - - val options = parser.parse(args : _*) - - if (options.has("help")) { - parser.printHelpOn(System.out) - Exit.exit(0) - } - - CommandLineUtils.checkRequiredArgs(parser, options, inFileOpt) - - val zkConnect = options.valueOf(zkConnectOpt) - val partitionOffsetFile = options.valueOf(inFileOpt) - - val zkUtils = ZkUtils(zkConnect, 30000, 30000, JaasUtils.isZkSecurityEnabled()) - val partitionOffsets: Map[String,String] = getPartitionOffsetsFromFile(partitionOffsetFile) - - updateZkOffsets(zkUtils, partitionOffsets) - } - - private def getPartitionOffsetsFromFile(filename: String):Map[String,String] = { - val br = new BufferedReader(new InputStreamReader(new FileInputStream(filename), StandardCharsets.UTF_8)) - try { - var partOffsetsMap: Map[String,String] = Map() - - var s: String = br.readLine() - while ( s != null && s.length() >= 1) { - val tokens = s.split(":") - - partOffsetsMap += tokens(0) -> tokens(1) - debug("adding node path [" + s + "]") - - s = br.readLine() - } - - partOffsetsMap - } finally { - br.close() - } - } - - private def updateZkOffsets(zkUtils: ZkUtils, partitionOffsets: Map[String,String]): Unit = { - for ((partition, offset) <- partitionOffsets) { - debug("updating [" + partition + "] with offset [" + offset + "]") - - try { - zkUtils.updatePersistentPath(partition, offset.toString) - } catch { - case e: Throwable => e.printStackTrace() - } - } - } -} diff --git a/core/src/main/scala/kafka/tools/JmxTool.scala b/core/src/main/scala/kafka/tools/JmxTool.scala index 27e46319e490a..c5303a9d96123 100644 --- a/core/src/main/scala/kafka/tools/JmxTool.scala +++ b/core/src/main/scala/kafka/tools/JmxTool.scala @@ -99,7 +99,7 @@ object JmxTool extends Logging { val url = new JMXServiceURL(options.valueOf(jmxServiceUrlOpt)) val interval = options.valueOf(reportingIntervalOpt).intValue - var oneTime = interval < 0 || options.has(oneTimeOpt) + val oneTime = interval < 0 || options.has(oneTimeOpt) val attributesWhitelistExists = options.has(attributesOpt) val attributesWhitelist = if(attributesWhitelistExists) Some(options.valueOf(attributesOpt).split(",").filterNot(_.equals(""))) else None val dateFormatExists = options.has(dateFormatOpt) diff --git a/core/src/main/scala/kafka/tools/MirrorMaker.scala b/core/src/main/scala/kafka/tools/MirrorMaker.scala index 907fe20f4141f..9cc6ebe1c86a8 100755 --- a/core/src/main/scala/kafka/tools/MirrorMaker.scala +++ b/core/src/main/scala/kafka/tools/MirrorMaker.scala @@ -25,26 +25,21 @@ import java.util.{Collections, Properties} import com.yammer.metrics.core.Gauge import joptsimple.OptionParser -import kafka.consumer.{BaseConsumer, BaseConsumerRecord, Blacklist, ConsumerIterator, ConsumerThreadId, ConsumerTimeoutException, TopicFilter, Whitelist, ZookeeperConsumerConnector, ConsumerConfig => OldConsumerConfig} -import kafka.javaapi.consumer.ConsumerRebalanceListener +import kafka.consumer.BaseConsumerRecord import kafka.metrics.KafkaMetricsGroup -import kafka.serializer.DefaultDecoder -import kafka.utils.{CommandLineUtils, CoreUtils, Logging, ZKConfig} -import org.apache.kafka.clients.consumer -import org.apache.kafka.clients.consumer.{CommitFailedException, Consumer, ConsumerRecord, KafkaConsumer, OffsetAndMetadata} +import kafka.utils.{CommandLineUtils, CoreUtils, Logging, Whitelist} +import org.apache.kafka.clients.consumer.{CommitFailedException, Consumer, ConsumerConfig, ConsumerRebalanceListener, ConsumerRecord, KafkaConsumer, OffsetAndMetadata} import org.apache.kafka.clients.producer.internals.ErrorLoggingCallback import org.apache.kafka.clients.producer.{KafkaProducer, ProducerConfig, ProducerRecord, RecordMetadata} -import org.apache.kafka.common.TopicPartition +import org.apache.kafka.common.{KafkaException, TopicPartition} import org.apache.kafka.common.serialization.ByteArrayDeserializer import org.apache.kafka.common.utils.Utils import org.apache.kafka.common.errors.WakeupException +import org.apache.kafka.common.record.RecordBatch import scala.collection.JavaConverters._ import scala.collection.mutable.HashMap import scala.util.control.ControlThrowable -import org.apache.kafka.clients.consumer.{ConsumerConfig => NewConsumerConfig} -import org.apache.kafka.common.header.internals.RecordHeaders -import org.apache.kafka.common.record.RecordBatch /** * The mirror maker has the following architecture: @@ -53,7 +48,7 @@ import org.apache.kafka.common.record.RecordBatch * - 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 + * 1. use producer with following settings * acks=all * retries=max integer * max.block.ms=max long @@ -95,8 +90,9 @@ object MirrorMaker extends Logging with KafkaMetricsGroup { .describedAs("config file") .ofType(classOf[String]) - val useNewConsumerOpt = parser.accepts("new.consumer", - "Use new consumer in mirror maker (this is the default).") + parser.accepts("new.consumer", + "DEPRECATED Use new consumer in mirror maker (this is the default so this option will be removed in " + + "a future version).") val producerConfigOpt = parser.accepts("producer.config", "Embedded producer config.") @@ -117,12 +113,6 @@ object MirrorMaker extends Logging with KafkaMetricsGroup { .describedAs("Java regex (String)") .ofType(classOf[String]) - val blacklistOpt = parser.accepts("blacklist", - "Blacklist of topics to mirror. Only old consumer supports blacklist.") - .withRequiredArg() - .describedAs("Java regex (String)") - .ofType(classOf[String]) - val offsetCommitIntervalMsOpt = parser.accepts("offset.commit.interval.ms", "Offset commit interval in ms.") .withRequiredArg() @@ -175,45 +165,19 @@ object MirrorMaker extends Logging with KafkaMetricsGroup { } CommandLineUtils.checkRequiredArgs(parser, options, consumerConfigOpt, producerConfigOpt) - val consumerProps = Utils.loadProps(options.valueOf(consumerConfigOpt)) - val useOldConsumer = consumerProps.containsKey(ZKConfig.ZkConnectProp) - - if (useOldConsumer) { - if (options.has(useNewConsumerOpt)) { - error(s"The consumer configuration parameter `${ZKConfig.ZkConnectProp}` is not valid when using --new.consumer") - sys.exit(1) - } - - if (consumerProps.containsKey(NewConsumerConfig.BOOTSTRAP_SERVERS_CONFIG)) { - error(s"The configuration parameters `${ZKConfig.ZkConnectProp}` (old consumer) and " + - s"`${NewConsumerConfig.BOOTSTRAP_SERVERS_CONFIG}` (new consumer) cannot be used together.") - sys.exit(1) - } - - if (List(whitelistOpt, blacklistOpt).count(options.has) != 1) { - error("Exactly one of whitelist or blacklist is required.") - sys.exit(1) - } - } else { - if (options.has(blacklistOpt)) { - error("blacklist can not be used when using new consumer in mirror maker. Use whitelist instead.") - sys.exit(1) - } - - if (!options.has(whitelistOpt)) { - error("whitelist must be specified when using new consumer in mirror maker.") - sys.exit(1) - } - - if (!consumerProps.containsKey(NewConsumerConfig.PARTITION_ASSIGNMENT_STRATEGY_CONFIG)) - System.err.println("WARNING: The default partition assignment strategy of the new-consumer-based mirror maker will " + - "change from 'range' to 'roundrobin' in an upcoming release (so that better load balancing can be achieved). If " + - "you prefer to make this switch in advance of that release add the following to the corresponding new-consumer " + - "config: 'partition.assignment.strategy=org.apache.kafka.clients.consumer.RoundRobinAssignor'") + if (!options.has(whitelistOpt)) { + error("whitelist must be specified") + sys.exit(1) } + if (!consumerProps.containsKey(ConsumerConfig.PARTITION_ASSIGNMENT_STRATEGY_CONFIG)) + System.err.println("WARNING: The default partition assignment strategy of the mirror maker will " + + "change from 'range' to 'roundrobin' in an upcoming release (so that better load balancing can be achieved). If " + + "you prefer to make this switch in advance of that release add the following to the corresponding " + + "config: 'partition.assignment.strategy=org.apache.kafka.clients.consumer.RoundRobinAssignor'") + abortOnSendFailure = options.valueOf(abortOnSendFailureOpt).toBoolean offsetCommitIntervalMs = options.valueOf(offsetCommitIntervalMsOpt).intValue() val numStreams = options.valueOf(numStreamsOpt).intValue() @@ -239,46 +203,23 @@ object MirrorMaker extends Logging with KafkaMetricsGroup { producer = new MirrorMakerProducer(sync, producerProps) // Create consumers - val mirrorMakerConsumers = if (useOldConsumer) { - val customRebalanceListener: Option[ConsumerRebalanceListener] = { - val customRebalanceListenerClass = options.valueOf(consumerRebalanceListenerOpt) - if (customRebalanceListenerClass != null) { - val rebalanceListenerArgs = options.valueOf(rebalanceListenerArgsOpt) - if (rebalanceListenerArgs != null) { - Some(CoreUtils.createObject[ConsumerRebalanceListener](customRebalanceListenerClass, rebalanceListenerArgs)) - } else { - Some(CoreUtils.createObject[ConsumerRebalanceListener](customRebalanceListenerClass)) - } - } else { - None - } - } - createOldConsumers( - numStreams, - consumerProps, - customRebalanceListener, - Option(options.valueOf(whitelistOpt)), - Option(options.valueOf(blacklistOpt))) - } else { - val customRebalanceListener: Option[org.apache.kafka.clients.consumer.ConsumerRebalanceListener] = { - val customRebalanceListenerClass = options.valueOf(consumerRebalanceListenerOpt) - if (customRebalanceListenerClass != null) { - val rebalanceListenerArgs = options.valueOf(rebalanceListenerArgsOpt) - if (rebalanceListenerArgs != null) { - Some(CoreUtils.createObject[org.apache.kafka.clients.consumer.ConsumerRebalanceListener](customRebalanceListenerClass, rebalanceListenerArgs)) - } else { - Some(CoreUtils.createObject[org.apache.kafka.clients.consumer.ConsumerRebalanceListener](customRebalanceListenerClass)) - } - } else { - None - } + val customRebalanceListener: Option[ConsumerRebalanceListener] = { + val customRebalanceListenerClass = options.valueOf(consumerRebalanceListenerOpt) + if (customRebalanceListenerClass != null) { + val rebalanceListenerArgs = options.valueOf(rebalanceListenerArgsOpt) + if (rebalanceListenerArgs != null) + Some(CoreUtils.createObject[ConsumerRebalanceListener](customRebalanceListenerClass, rebalanceListenerArgs)) + else + Some(CoreUtils.createObject[ConsumerRebalanceListener](customRebalanceListenerClass)) + } else { + None } - createNewConsumers( - numStreams, - consumerProps, - customRebalanceListener, - Option(options.valueOf(whitelistOpt))) } + val mirrorMakerConsumers = createConsumers( + numStreams, + consumerProps, + customRebalanceListener, + Option(options.valueOf(whitelistOpt))) // Create mirror maker threads. mirrorMakerThreads = (0 until numStreams) map (i => @@ -307,43 +248,10 @@ object MirrorMaker extends Logging with KafkaMetricsGroup { mirrorMakerThreads.foreach(_.awaitShutdown()) } - private def createOldConsumers(numStreams: Int, - consumerConfigProps: Properties, - customRebalanceListener: Option[ConsumerRebalanceListener], - whitelist: Option[String], - blacklist: Option[String]) : Seq[MirrorMakerBaseConsumer] = { - // 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") - val connectors = (0 until numStreams) map { i => - consumerConfigProps.setProperty("client.id", groupIdString + "-" + i.toString) - val consumerConfig = new OldConsumerConfig(consumerConfigProps) - new ZookeeperConsumerConnector(consumerConfig) - } - - // create filters - val filterSpec = if (whitelist.isDefined) - new Whitelist(whitelist.get) - else if (blacklist.isDefined) - new Blacklist(blacklist.get) - else - throw new IllegalArgumentException("Either whitelist or blacklist should be defined!") - (0 until numStreams) map { i => - val consumer = new MirrorMakerOldConsumer(connectors(i), filterSpec) - val consumerRebalanceListener = new InternalRebalanceListenerForOldConsumer(consumer, customRebalanceListener) - connectors(i).setConsumerRebalanceListener(consumerRebalanceListener) - consumer - } - } - - def createNewConsumers(numStreams: Int, - consumerConfigProps: Properties, - customRebalanceListener: Option[org.apache.kafka.clients.consumer.ConsumerRebalanceListener], - whitelist: Option[String]) : Seq[MirrorMakerBaseConsumer] = { + def createConsumers(numStreams: Int, + consumerConfigProps: Properties, + customRebalanceListener: Option[ConsumerRebalanceListener], + whitelist: Option[String]): Seq[ConsumerWrapper] = { // Disable consumer auto offsets commit to prevent data loss. maybeSetDefaultProperty(consumerConfigProps, "enable.auto.commit", "false") // Hardcode the deserializer to ByteArrayDeserializer @@ -355,28 +263,28 @@ object MirrorMaker extends Logging with KafkaMetricsGroup { consumerConfigProps.setProperty("client.id", groupIdString + "-" + i.toString) new KafkaConsumer[Array[Byte], Array[Byte]](consumerConfigProps) } - whitelist.getOrElse(throw new IllegalArgumentException("White list cannot be empty for new consumer")) - consumers.map(consumer => new MirrorMakerNewConsumer(consumer, customRebalanceListener, whitelist)) + whitelist.getOrElse(throw new IllegalArgumentException("White list cannot be empty")) + consumers.map(consumer => new ConsumerWrapper(consumer, customRebalanceListener, whitelist)) } - def commitOffsets(mirrorMakerConsumer: MirrorMakerBaseConsumer) { + def commitOffsets(consumerWrapper: ConsumerWrapper) { if (!exitingOnSendFailure) { trace("Committing offsets.") try { - mirrorMakerConsumer.commit() + consumerWrapper.commit() } catch { case e: WakeupException => // we only call wakeup() once to close the consumer, // so if we catch it in commit we can safely retry // and re-throw to break the loop - mirrorMakerConsumer.commit() + consumerWrapper.commit() throw e case _: CommitFailedException => warn("Failed to commit offsets because the consumer group has rebalanced and assigned partitions to " + "another instance. If you see this regularly, it could indicate that you need to either increase " + - s"the consumer's ${consumer.ConsumerConfig.SESSION_TIMEOUT_MS_CONFIG} or reduce the number of records " + - s"handled on each iteration with ${consumer.ConsumerConfig.MAX_POLL_RECORDS_CONFIG}") + s"the consumer's ${ConsumerConfig.SESSION_TIMEOUT_MS_CONFIG} or reduce the number of records " + + s"handled on each iteration with ${ConsumerConfig.MAX_POLL_RECORDS_CONFIG}") } } else { info("Exiting on send failure, skip committing offsets.") @@ -405,7 +313,7 @@ object MirrorMaker extends Logging with KafkaMetricsGroup { info("Property %s is overridden to %s - data loss or message reordering is possible.".format(propertyName, propertyValue)) } - class MirrorMakerThread(mirrorMakerConsumer: MirrorMakerBaseConsumer, + class MirrorMakerThread(consumerWrapper: ConsumerWrapper, val threadId: Int) extends Thread with Logging with KafkaMetricsGroup { private val threadName = "mirrormaker-thread-" + threadId private val shutdownLatch: CountDownLatch = new CountDownLatch(1) @@ -415,31 +323,42 @@ object MirrorMaker extends Logging with KafkaMetricsGroup { setName(threadName) + private def toBaseConsumerRecord(record: ConsumerRecord[Array[Byte], Array[Byte]]): BaseConsumerRecord = + BaseConsumerRecord(record.topic, + record.partition, + record.offset, + record.timestamp, + record.timestampType, + record.key, + record.value, + record.headers) + override def run() { info("Starting mirror maker thread " + threadName) try { - mirrorMakerConsumer.init() + consumerWrapper.init() - // We need the two while loop to make sure when old consumer is used, even there is no message we - // still commit offset. When new consumer is used, this is handled by poll(timeout). + // We needed two while loops due to the old consumer semantics, this can now be simplified while (!exitingOnSendFailure && !shuttingDown) { try { - while (!exitingOnSendFailure && !shuttingDown && mirrorMakerConsumer.hasData) { - val data = mirrorMakerConsumer.receive() + while (!exitingOnSendFailure && !shuttingDown) { + val data = consumerWrapper.receive() if (data.value != null) { trace("Sending message with value size %d and offset %d.".format(data.value.length, data.offset)) } else { trace("Sending message with null value and offset %d.".format(data.offset)) } - val records = messageHandler.handle(data) + val records = messageHandler.handle(toBaseConsumerRecord(data)) records.asScala.foreach(producer.send) maybeFlushAndCommitOffsets() } } catch { - case _: ConsumerTimeoutException => - trace("Caught ConsumerTimeoutException, continue iteration.") + case _: NoRecordsException => + trace("Caught NoRecordsException, continue iteration.") case _: WakeupException => - trace("Caught ConsumerWakeupException, continue iteration.") + trace("Caught WakeupException, continue iteration.") + case e: KafkaException if (shuttingDown || exitingOnSendFailure) => + trace(s"Ignoring caught KafkaException during shutdown. sendFailure: $exitingOnSendFailure.", e) } maybeFlushAndCommitOffsets() } @@ -454,12 +373,12 @@ object MirrorMaker extends Logging with KafkaMetricsGroup { // note that this commit is skipped if flush() fails which ensures that we don't lose messages info("Committing consumer offsets.") - commitOffsets(mirrorMakerConsumer) + commitOffsets(consumerWrapper) }, this) info("Shutting down consumer connectors.") - CoreUtils.swallow(mirrorMakerConsumer.stop(), this) - CoreUtils.swallow(mirrorMakerConsumer.cleanup(), this) + CoreUtils.swallow(consumerWrapper.wakeup(), this) + CoreUtils.swallow(consumerWrapper.close(), this) shutdownLatch.countDown() info("Mirror maker thread stopped") // if it exits accidentally, stop the entire mirror maker @@ -471,14 +390,11 @@ object MirrorMaker extends Logging with KafkaMetricsGroup { } def maybeFlushAndCommitOffsets() { - val commitRequested = mirrorMakerConsumer.commitRequested() - if (commitRequested || System.currentTimeMillis() - lastOffsetCommitMs > offsetCommitIntervalMs) { + if (System.currentTimeMillis() - lastOffsetCommitMs > offsetCommitIntervalMs) { debug("Committing MirrorMaker state.") producer.flush() - commitOffsets(mirrorMakerConsumer) + commitOffsets(consumerWrapper) lastOffsetCommitMs = System.currentTimeMillis() - if (commitRequested) - mirrorMakerConsumer.notifyCommit() } } @@ -486,7 +402,7 @@ object MirrorMaker extends Logging with KafkaMetricsGroup { try { info(threadName + " shutting down") shuttingDown = true - mirrorMakerConsumer.stop() + consumerWrapper.wakeup() } catch { case _: InterruptedException => @@ -505,107 +421,19 @@ object MirrorMaker extends Logging with KafkaMetricsGroup { } } - private[kafka] trait MirrorMakerBaseConsumer extends BaseConsumer { - def init() - def commitRequested(): Boolean - def notifyCommit() - def requestAndWaitForCommit() - def hasData : Boolean - } - - private class MirrorMakerOldConsumer(connector: ZookeeperConsumerConnector, - filterSpec: TopicFilter) extends MirrorMakerBaseConsumer { - private var iter: ConsumerIterator[Array[Byte], Array[Byte]] = null - private var immediateCommitRequested: Boolean = false - private var numCommitsNotified: Long = 0 - - override def init() { - // Creating one stream per each connector instance - val streams = connector.createMessageStreamsByFilter(filterSpec, 1, new DefaultDecoder(), new DefaultDecoder()) - require(streams.size == 1) - val stream = streams.head - iter = stream.iterator() - } - - override def requestAndWaitForCommit() { - this.synchronized { - // only wait() if mirrorMakerConsumer has been initialized and it has not been cleaned up. - if (iter != null) { - immediateCommitRequested = true - val nextNumCommitsNotified = numCommitsNotified + 1 - do { - this.wait() - } while (numCommitsNotified < nextNumCommitsNotified) - } - } - } - - override def notifyCommit() { - this.synchronized { - immediateCommitRequested = false - numCommitsNotified = numCommitsNotified + 1 - this.notifyAll() - } - } - - override def commitRequested(): Boolean = { - this.synchronized { - immediateCommitRequested - } - } - - override def hasData = iter.hasNext() - - override def receive() : BaseConsumerRecord = { - val messageAndMetadata = iter.next() - BaseConsumerRecord(messageAndMetadata.topic, - messageAndMetadata.partition, - messageAndMetadata.offset, - messageAndMetadata.timestamp, - messageAndMetadata.timestampType, - messageAndMetadata.key, - messageAndMetadata.message, - new RecordHeaders()) - } - - override def stop() { - // Do nothing - } - - override def cleanup() { - // We need to set the iterator to null and notify the rebalance listener thread. - // This is to handle the case that the consumer rebalance is triggered when the - // mirror maker thread is shutting down and the rebalance listener is waiting for the offset commit. - this.synchronized { - iter = null - if (immediateCommitRequested) { - notifyCommit() - } - } - connector.shutdown() - } - - override def commit() { - connector.commitOffsets - } - } - - // Only for testing - private[tools] class MirrorMakerNewConsumer(consumer: Consumer[Array[Byte], Array[Byte]], - customRebalanceListener: Option[org.apache.kafka.clients.consumer.ConsumerRebalanceListener], - whitelistOpt: Option[String]) - extends MirrorMakerBaseConsumer { + // Visible for testing + private[tools] class ConsumerWrapper(consumer: Consumer[Array[Byte], Array[Byte]], + customRebalanceListener: Option[ConsumerRebalanceListener], + whitelistOpt: Option[String]) { val regex = whitelistOpt.getOrElse(throw new IllegalArgumentException("New consumer only supports whitelist.")) var recordIter: java.util.Iterator[ConsumerRecord[Array[Byte], Array[Byte]]] = null - // TODO: we need to manually maintain the consumed offsets for new consumer - // since its internal consumed position is updated in batch rather than one - // record at a time, this can be resolved when we break the unification of both consumers + // We manually maintain the consumed offsets for historical reasons and it could be simplified private val offsets = new HashMap[TopicPartition, Long]() - override def init() { - debug("Initiating new consumer") - val consumerRebalanceListener = new InternalRebalanceListenerForNewConsumer(this, customRebalanceListener) + def init() { + debug("Initiating consumer") + val consumerRebalanceListener = new InternalRebalanceListener(this, customRebalanceListener) whitelistOpt.foreach { whitelist => try { consumer.subscribe(Pattern.compile(Whitelist(whitelist).regex), consumerRebalanceListener) @@ -617,21 +445,7 @@ object MirrorMaker extends Logging with KafkaMetricsGroup { } } - override def requestAndWaitForCommit() { - // Do nothing - } - - override def notifyCommit() { - // Do nothing - } - - override def commitRequested(): Boolean = { - false - } - - override def hasData = true - - override def receive() : BaseConsumerRecord = { + def receive(): ConsumerRecord[Array[Byte], Array[Byte]] = { if (recordIter == null || !recordIter.hasNext) { // In scenarios where data does not arrive within offsetCommitIntervalMs and // offsetCommitIntervalMs is less than poll's timeout, offset commit will be delayed for any @@ -640,67 +454,42 @@ object MirrorMaker extends Logging with KafkaMetricsGroup { // commit. recordIter = consumer.poll(1000).iterator if (!recordIter.hasNext) - throw new ConsumerTimeoutException + throw new NoRecordsException } val record = recordIter.next() val tp = new TopicPartition(record.topic, record.partition) offsets.put(tp, record.offset + 1) - - BaseConsumerRecord(record.topic, - record.partition, - record.offset, - record.timestamp, - record.timestampType, - record.key, - record.value, - record.headers) + record } - override def stop() { + def wakeup() { consumer.wakeup() } - override def cleanup() { + def close() { consumer.close() } - override def commit() { + def commit() { consumer.commitSync(offsets.map { case (tp, offset) => (tp, new OffsetAndMetadata(offset, ""))}.asJava) offsets.clear() } } - private class InternalRebalanceListenerForNewConsumer(mirrorMakerConsumer: MirrorMakerBaseConsumer, - customRebalanceListenerForNewConsumer: Option[org.apache.kafka.clients.consumer.ConsumerRebalanceListener]) - extends org.apache.kafka.clients.consumer.ConsumerRebalanceListener { + private class InternalRebalanceListener(consumerWrapper: ConsumerWrapper, + customRebalanceListener: Option[ConsumerRebalanceListener]) + extends ConsumerRebalanceListener { override def onPartitionsRevoked(partitions: util.Collection[TopicPartition]) { producer.flush() - commitOffsets(mirrorMakerConsumer) - customRebalanceListenerForNewConsumer.foreach(_.onPartitionsRevoked(partitions)) + commitOffsets(consumerWrapper) + customRebalanceListener.foreach(_.onPartitionsRevoked(partitions)) } override def onPartitionsAssigned(partitions: util.Collection[TopicPartition]) { - customRebalanceListenerForNewConsumer.foreach(_.onPartitionsAssigned(partitions)) - } - } - - private class InternalRebalanceListenerForOldConsumer(mirrorMakerConsumer: MirrorMakerBaseConsumer, - customRebalanceListenerForOldConsumer: Option[ConsumerRebalanceListener]) - extends ConsumerRebalanceListener { - - override def beforeReleasingPartitions(partitionOwnership: java.util.Map[String, java.util.Set[java.lang.Integer]]) { - // The zookeeper listener thread, which executes this method, needs to wait for MirrorMakerThread to flush data and commit offset - mirrorMakerConsumer.requestAndWaitForCommit() - // invoke custom consumer rebalance listener - customRebalanceListenerForOldConsumer.foreach(_.beforeReleasingPartitions(partitionOwnership)) - } - - override def beforeStartingFetchers(consumerId: String, - partitionAssignment: java.util.Map[String, java.util.Map[java.lang.Integer, ConsumerThreadId]]) { - customRebalanceListenerForOldConsumer.foreach(_.beforeStartingFetchers(consumerId, partitionAssignment)) + customRebalanceListener.foreach(_.onPartitionsAssigned(partitions)) } } @@ -763,4 +552,6 @@ object MirrorMaker extends Logging with KafkaMetricsGroup { } } + private class NoRecordsException extends RuntimeException + } diff --git a/core/src/main/scala/kafka/tools/ReplicaVerificationTool.scala b/core/src/main/scala/kafka/tools/ReplicaVerificationTool.scala index 0408e9212a3d5..1ecea09b9946d 100644 --- a/core/src/main/scala/kafka/tools/ReplicaVerificationTool.scala +++ b/core/src/main/scala/kafka/tools/ReplicaVerificationTool.scala @@ -17,25 +17,35 @@ package kafka.tools +import java.net.SocketTimeoutException import java.text.SimpleDateFormat -import java.util.Date +import java.util import java.util.concurrent.CountDownLatch -import java.util.concurrent.atomic.AtomicReference +import java.util.concurrent.atomic.{AtomicInteger, AtomicReference} import java.util.regex.{Pattern, PatternSyntaxException} +import java.util.{Date, Properties} import joptsimple.OptionParser import kafka.api._ -import kafka.client.ClientUtils -import kafka.cluster.BrokerEndPoint -import kafka.common.TopicAndPartition -import kafka.consumer.{ConsumerConfig, SimpleConsumer, Whitelist} -import kafka.message.{ByteBufferMessageSet, MessageSet} +import kafka.utils.Whitelist import kafka.utils._ -import org.apache.kafka.common.protocol.Errors -import org.apache.kafka.common.utils.Time +import org.apache.kafka.clients._ +import org.apache.kafka.clients.admin.{ListTopicsOptions, TopicDescription} +import org.apache.kafka.clients.consumer.{ConsumerConfig, KafkaConsumer} +import org.apache.kafka.common.metrics.Metrics +import org.apache.kafka.common.network.{NetworkReceive, Selectable, Selector} +import org.apache.kafka.common.protocol.{ApiKeys, Errors} +import org.apache.kafka.common.record.MemoryRecords +import org.apache.kafka.common.requests.AbstractRequest.Builder +import org.apache.kafka.common.requests.{AbstractRequest, FetchResponse, ListOffsetRequest, FetchRequest => JFetchRequest} +import org.apache.kafka.common.serialization.StringDeserializer +import org.apache.kafka.common.utils.{LogContext, Time} +import org.apache.kafka.common.{Node, TopicPartition} + +import scala.collection.JavaConverters._ /** - * For verifying the consistency among replicas. + * For verifying the consistency among replicas. * * 1. start a fetcher on every broker. * 2. each fetcher does the following @@ -44,11 +54,11 @@ import org.apache.kafka.common.utils.Time * 2.3 waits for all other fetchers to finish step 2.2 * 2.4 one of the fetchers verifies the consistency of fetched results among replicas * - * The consistency verification is up to the high watermark. The tool reports the - * max lag between the verified offset and the high watermark among all partitions. + * The consistency verification is up to the high watermark. The tool reports the + * max lag between the verified offset and the high watermark among all partitions. * - * If a broker goes down, the verification of the partitions on that broker is delayed - * until the broker is up again. + * If a broker goes down, the verification of the partitions on that broker is delayed + * until the broker is up again. * * Caveats: * 1. The tools needs all brokers to be up at startup time. @@ -56,7 +66,7 @@ import org.apache.kafka.common.utils.Time */ object ReplicaVerificationTool extends Logging { - val clientId= "replicaVerificationTool" + val clientId = "replicaVerificationTool" val dateFormatString = "yyyy-MM-dd HH:mm:ss,SSS" val dateFormat = new SimpleDateFormat(dateFormatString) @@ -74,7 +84,7 @@ object ReplicaVerificationTool extends Logging { .withRequiredArg .describedAs("bytes") .ofType(classOf[java.lang.Integer]) - .defaultsTo(ConsumerConfig.FetchSize) + .defaultsTo(ConsumerConfig.DEFAULT_MAX_PARTITION_FETCH_BYTES) val maxWaitMsOpt = parser.accepts("max-wait-ms", "The max amount of time each fetch request waits.") .withRequiredArg .describedAs("ms") @@ -96,18 +106,16 @@ object ReplicaVerificationTool extends Logging { .ofType(classOf[java.lang.Long]) .defaultsTo(30 * 1000L) - if(args.length == 0) + if (args.length == 0) CommandLineUtils.printUsageAndDie(parser, "Validate that all replicas for a set of topics have the same data.") - val options = parser.parse(args : _*) + val options = parser.parse(args: _*) CommandLineUtils.checkRequiredArgs(parser, options, brokerListOpt) val regex = options.valueOf(topicWhiteListOpt) val topicWhiteListFiler = new Whitelist(regex) - try { - Pattern.compile(regex) - } + try Pattern.compile(regex) catch { case _: PatternSyntaxException => throw new RuntimeException(regex + " is an invalid regex.") @@ -120,68 +128,68 @@ object ReplicaVerificationTool extends Logging { // getting topic metadata info("Getting topic metadata...") val brokerList = options.valueOf(brokerListOpt) - ToolsUtils.validatePortOrDie(parser,brokerList) - val metadataTargetBrokers = ClientUtils.parseBrokerList(brokerList) - val topicsMetadataResponse = ClientUtils.fetchTopicMetadata(Set[String](), metadataTargetBrokers, clientId, maxWaitMs) - val brokerMap = topicsMetadataResponse.brokers.map(b => (b.id, b)).toMap - val filteredTopicMetadata = topicsMetadataResponse.topicsMetadata.filter( - topicMetadata => if (topicWhiteListFiler.isTopicAllowed(topicMetadata.topic, excludeInternalTopics = false)) - true - else - false - ) + ToolsUtils.validatePortOrDie(parser, brokerList) + + val (topicsMetadata, brokerInfo) = { + val adminClient = createAdminClient(brokerList) + try ((listTopicsMetadata(adminClient), brokerDetails(adminClient))) + finally CoreUtils.swallow(adminClient.close(), this) + } + + val filteredTopicMetadata = topicsMetadata.filter { topicMetaData => + topicWhiteListFiler.isTopicAllowed(topicMetaData.name, excludeInternalTopics = false) + } if (filteredTopicMetadata.isEmpty) { - error("No topics found. " + topicWhiteListOpt + ", if specified, is either filtering out all topics or there is no topic.") + error(s"No topics found. $topicWhiteListOpt if specified, is either filtering out all topics or there is no topic.") Exit.exit(1) } - val topicPartitionReplicaList: Seq[TopicPartitionReplica] = filteredTopicMetadata.flatMap( - topicMetadataResponse => - topicMetadataResponse.partitionsMetadata.flatMap( - partitionMetadata => - partitionMetadata.replicas.map(broker => - TopicPartitionReplica(topic = topicMetadataResponse.topic, partitionId = partitionMetadata.partitionId, replicaId = broker.id)) - ) - ) - debug("Selected topic partitions: " + topicPartitionReplicaList) - val topicAndPartitionsPerBroker: Map[Int, Seq[TopicAndPartition]] = topicPartitionReplicaList.groupBy(_.replicaId) - .map { case (brokerId, partitions) => - brokerId -> partitions.map { partition => TopicAndPartition(partition.topic, partition.partitionId) } } - debug("Topic partitions per broker: " + topicAndPartitionsPerBroker) - val expectedReplicasPerTopicAndPartition: Map[TopicAndPartition, Int] = - topicPartitionReplicaList.groupBy(replica => TopicAndPartition(replica.topic, replica.partitionId)) - .map { case (topicAndPartition, replicaSet) => topicAndPartition -> replicaSet.size } - debug("Expected replicas per topic partition: " + expectedReplicasPerTopicAndPartition) - val leadersPerBroker: Map[Int, Seq[TopicAndPartition]] = filteredTopicMetadata.flatMap { topicMetadataResponse => - topicMetadataResponse.partitionsMetadata.map { partitionMetadata => - (TopicAndPartition(topicMetadataResponse.topic, partitionMetadata.partitionId), partitionMetadata.leader.get.id) + val topicPartitionReplicas = filteredTopicMetadata.flatMap { topicMetadata => + topicMetadata.partitions.asScala.flatMap { partitionMetadata => + partitionMetadata.replicas.asScala.map { node => + TopicPartitionReplica(topic = topicMetadata.name, partitionId = partitionMetadata.partition, replicaId = node.id) + } + } + } + debug(s"Selected topic partitions: $topicPartitionReplicas") + val brokerToTopicPartitions = topicPartitionReplicas.groupBy(_.replicaId).map { case (brokerId, partitions) => + brokerId -> partitions.map { partition => new TopicPartition(partition.topic, partition.partitionId) } + } + debug(s"Topic partitions per broker: $brokerToTopicPartitions") + val expectedReplicasPerTopicPartition = topicPartitionReplicas.groupBy { replica => + new TopicPartition(replica.topic, replica.partitionId) + }.map { case (topicAndPartition, replicaSet) => topicAndPartition -> replicaSet.size } + debug(s"Expected replicas per topic partition: $expectedReplicasPerTopicPartition") + + val topicPartitions = filteredTopicMetadata.flatMap { topicMetaData => + topicMetaData.partitions.asScala.map { partitionMetadata => + new TopicPartition(topicMetaData.name, partitionMetadata.partition) } - }.groupBy(_._2).mapValues(topicAndPartitionAndLeaderIds => topicAndPartitionAndLeaderIds.map { case (topicAndPartition, _) => - topicAndPartition - }) - debug("Leaders per broker: " + leadersPerBroker) - - val replicaBuffer = new ReplicaBuffer(expectedReplicasPerTopicAndPartition, - leadersPerBroker, - topicAndPartitionsPerBroker.size, - brokerMap, - initialOffsetTime, - reportInterval) + } + + val consumerProps = consumerConfig(brokerList) + + val replicaBuffer = new ReplicaBuffer(expectedReplicasPerTopicPartition, + initialOffsets(topicPartitions, consumerProps, initialOffsetTime), + brokerToTopicPartitions.size, + reportInterval) // create all replica fetcher threads - val verificationBrokerId = topicAndPartitionsPerBroker.head._1 - val fetcherThreads: Iterable[ReplicaFetcher] = topicAndPartitionsPerBroker.map { - case (brokerId, topicAndPartitions) => - new ReplicaFetcher(name = "ReplicaFetcher-" + brokerId, - sourceBroker = brokerMap(brokerId), - topicAndPartitions = topicAndPartitions, - replicaBuffer = replicaBuffer, - socketTimeout = 30000, - socketBufferSize = 256000, - fetchSize = fetchSize, - maxWait = maxWaitMs, - minBytes = 1, - doVerification = brokerId == verificationBrokerId) + val verificationBrokerId = brokerToTopicPartitions.head._1 + val counter = new AtomicInteger(0) + val fetcherThreads: Iterable[ReplicaFetcher] = brokerToTopicPartitions.map { case (brokerId, topicPartitions) => + new ReplicaFetcher(name = s"ReplicaFetcher-$brokerId", + sourceBroker = brokerInfo(brokerId), + topicPartitions = topicPartitions, + replicaBuffer = replicaBuffer, + socketTimeout = 30000, + socketBufferSize = 256000, + fetchSize = fetchSize, + maxWait = maxWaitMs, + minBytes = 1, + doVerification = brokerId == verificationBrokerId, + consumerProps, + fetcherId = counter.incrementAndGet()) } Runtime.getRuntime.addShutdownHook(new Thread() { @@ -194,87 +202,112 @@ object ReplicaVerificationTool extends Logging { println(ReplicaVerificationTool.getCurrentTimeString() + ": verification process is started.") } + + private def listTopicsMetadata(adminClient: admin.AdminClient): Seq[TopicDescription] = { + val topics = adminClient.listTopics(new ListTopicsOptions().listInternal(true)).names.get + adminClient.describeTopics(topics).all.get.values.asScala.toBuffer + } + + private def brokerDetails(adminClient: admin.AdminClient): Map[Int, Node] = { + adminClient.describeCluster.nodes.get.asScala.map(n => (n.id, n)).toMap + } + + private def createAdminClient(brokerUrl: String): admin.AdminClient = { + val props = new Properties() + props.put(CommonClientConfigs.BOOTSTRAP_SERVERS_CONFIG, brokerUrl) + admin.AdminClient.create(props) + } + + private def initialOffsets(topicPartitions: Seq[TopicPartition], consumerConfig: Properties, + initialOffsetTime: Long): Map[TopicPartition, Long] = { + val consumer = createConsumer(consumerConfig) + try { + if (ListOffsetRequest.LATEST_TIMESTAMP == initialOffsetTime) + consumer.endOffsets(topicPartitions.asJava).asScala.mapValues(_.longValue).toMap + else if (ListOffsetRequest.EARLIEST_TIMESTAMP == initialOffsetTime) + consumer.beginningOffsets(topicPartitions.asJava).asScala.mapValues(_.longValue).toMap + else { + val timestampsToSearch = topicPartitions.map(tp => tp -> (initialOffsetTime: java.lang.Long)).toMap + consumer.offsetsForTimes(timestampsToSearch.asJava).asScala.mapValues(v => v.offset).toMap + } + } finally consumer.close() + } + + private def consumerConfig(brokerUrl: String): Properties = { + val properties = new Properties() + properties.put(ConsumerConfig.BOOTSTRAP_SERVERS_CONFIG, brokerUrl) + properties.put(ConsumerConfig.GROUP_ID_CONFIG, "ReplicaVerification") + properties.put(ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG, classOf[StringDeserializer]) + properties.put(ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG, classOf[StringDeserializer]) + properties + } + + private def createConsumer(consumerConfig: Properties): KafkaConsumer[String, String] = + new KafkaConsumer(consumerConfig) } -private case class TopicPartitionReplica(topic: String, partitionId: Int, replicaId: Int) +private case class TopicPartitionReplica(topic: String, partitionId: Int, replicaId: Int) private case class MessageInfo(replicaId: Int, offset: Long, nextOffset: Long, checksum: Long) -private class ReplicaBuffer(expectedReplicasPerTopicAndPartition: Map[TopicAndPartition, Int], - leadersPerBroker: Map[Int, Seq[TopicAndPartition]], +private class ReplicaBuffer(expectedReplicasPerTopicPartition: Map[TopicPartition, Int], + initialOffsets: Map[TopicPartition, Long], expectedNumFetchers: Int, - brokerMap: Map[Int, BrokerEndPoint], - initialOffsetTime: Long, reportInterval: Long) extends Logging { - private val fetchOffsetMap = new Pool[TopicAndPartition, Long] - private val messageSetCache = new Pool[TopicAndPartition, Pool[Int, FetchResponsePartitionData]] + private val fetchOffsetMap = new Pool[TopicPartition, Long] + private val recordsCache = new Pool[TopicPartition, Pool[Int, FetchResponse.PartitionData[MemoryRecords]]] private val fetcherBarrier = new AtomicReference(new CountDownLatch(expectedNumFetchers)) private val verificationBarrier = new AtomicReference(new CountDownLatch(1)) @volatile private var lastReportTime = Time.SYSTEM.milliseconds private var maxLag: Long = -1L private var offsetWithMaxLag: Long = -1L - private var maxLagTopicAndPartition: TopicAndPartition = null + private var maxLagTopicAndPartition: TopicPartition = null initialize() def createNewFetcherBarrier() { fetcherBarrier.set(new CountDownLatch(expectedNumFetchers)) } - def getFetcherBarrier() = fetcherBarrier.get() + def getFetcherBarrier() = fetcherBarrier.get def createNewVerificationBarrier() { verificationBarrier.set(new CountDownLatch(1)) } - def getVerificationBarrier() = verificationBarrier.get() + def getVerificationBarrier() = verificationBarrier.get private def initialize() { - for (topicAndPartition <- expectedReplicasPerTopicAndPartition.keySet) - messageSetCache.put(topicAndPartition, new Pool[Int, FetchResponsePartitionData]) + for (topicPartition <- expectedReplicasPerTopicPartition.keySet) + recordsCache.put(topicPartition, new Pool[Int, FetchResponse.PartitionData[MemoryRecords]]) setInitialOffsets() } - private def offsetResponseStringWithError(offsetResponse: OffsetResponse): String = { - offsetResponse.partitionErrorAndOffsets.filter { case (_, partitionOffsetsResponse) => - partitionOffsetsResponse.error != Errors.NONE - }.mkString - } private def setInitialOffsets() { - for ((brokerId, topicAndPartitions) <- leadersPerBroker) { - val broker = brokerMap(brokerId) - val consumer = new SimpleConsumer(broker.host, broker.port, 10000, 100000, ReplicaVerificationTool.clientId) - val initialOffsetMap: Map[TopicAndPartition, PartitionOffsetRequestInfo] = - topicAndPartitions.map(topicAndPartition => topicAndPartition -> PartitionOffsetRequestInfo(initialOffsetTime, 1)).toMap - val offsetRequest = OffsetRequest(initialOffsetMap) - val offsetResponse = consumer.getOffsetsBefore(offsetRequest) - assert(!offsetResponse.hasError, offsetResponseStringWithError(offsetResponse)) - offsetResponse.partitionErrorAndOffsets.foreach { case (topicAndPartition, partitionOffsetResponse) => - fetchOffsetMap.put(topicAndPartition, partitionOffsetResponse.offsets.head) - } - } + for ((tp, offset) <- initialOffsets) + fetchOffsetMap.put(tp, offset) } - def addFetchedData(topicAndPartition: TopicAndPartition, replicaId: Int, partitionData: FetchResponsePartitionData) { - messageSetCache.get(topicAndPartition).put(replicaId, partitionData) + def addFetchedData(topicAndPartition: TopicPartition, replicaId: Int, partitionData: FetchResponse.PartitionData[MemoryRecords]) { + recordsCache.get(topicAndPartition).put(replicaId, partitionData) } - def getOffset(topicAndPartition: TopicAndPartition) = { + def getOffset(topicAndPartition: TopicPartition) = { fetchOffsetMap.get(topicAndPartition) } def verifyCheckSum(println: String => Unit) { debug("Begin verification") maxLag = -1L - for ((topicAndPartition, fetchResponsePerReplica) <- messageSetCache) { - debug("Verifying " + topicAndPartition) - assert(fetchResponsePerReplica.size == expectedReplicasPerTopicAndPartition(topicAndPartition), - "fetched " + fetchResponsePerReplica.size + " replicas for " + topicAndPartition + ", but expected " - + expectedReplicasPerTopicAndPartition(topicAndPartition) + " replicas") + for ((topicPartition, fetchResponsePerReplica) <- recordsCache) { + debug("Verifying " + topicPartition) + assert(fetchResponsePerReplica.size == expectedReplicasPerTopicPartition(topicPartition), + "fetched " + fetchResponsePerReplica.size + " replicas for " + topicPartition + ", but expected " + + expectedReplicasPerTopicPartition(topicPartition) + " replicas") val recordBatchIteratorMap = fetchResponsePerReplica.map { case (replicaId, fetchResponse) => - replicaId -> fetchResponse.messages.asInstanceOf[ByteBufferMessageSet].asRecords.batches.iterator + replicaId -> fetchResponse.records.batches.iterator } - val maxHw = fetchResponsePerReplica.values.map(_.hw).max + val maxHw = fetchResponsePerReplica.values.map(_.highWatermark).max // Iterate one message at a time from every replica, until high watermark is reached. var isMessageInAllReplicas = true @@ -286,7 +319,7 @@ private class ReplicaBuffer(expectedReplicasPerTopicAndPartition: Map[TopicAndPa val batch = recordBatchIterator.next() // only verify up to the high watermark - if (batch.lastOffset >= fetchResponsePerReplica.get(replicaId).hw) + if (batch.lastOffset >= fetchResponsePerReplica.get(replicaId).highWatermark) isMessageInAllReplicas = false else { messageInfoFromFirstReplicaOpt match { @@ -295,7 +328,7 @@ private class ReplicaBuffer(expectedReplicasPerTopicAndPartition: Map[TopicAndPa MessageInfo(replicaId, batch.lastOffset, batch.nextOffset, batch.checksum)) case Some(messageInfoFromFirstReplica) => if (messageInfoFromFirstReplica.offset != batch.lastOffset) { - println(ReplicaVerificationTool.getCurrentTimeString + ": partition " + topicAndPartition + println(ReplicaVerificationTool.getCurrentTimeString + ": partition " + topicPartition + ": replica " + messageInfoFromFirstReplica.replicaId + "'s offset " + messageInfoFromFirstReplica.offset + " doesn't match replica " + replicaId + "'s offset " + batch.lastOffset) @@ -303,7 +336,7 @@ private class ReplicaBuffer(expectedReplicasPerTopicAndPartition: Map[TopicAndPa } if (messageInfoFromFirstReplica.checksum != batch.checksum) println(ReplicaVerificationTool.getCurrentTimeString + ": partition " - + topicAndPartition + " has unmatched checksum at offset " + batch.lastOffset + "; replica " + + topicPartition + " has unmatched checksum at offset " + batch.lastOffset + "; replica " + messageInfoFromFirstReplica.replicaId + "'s checksum " + messageInfoFromFirstReplica.checksum + "; replica " + replicaId + "'s checksum " + batch.checksum) } @@ -313,20 +346,20 @@ private class ReplicaBuffer(expectedReplicasPerTopicAndPartition: Map[TopicAndPa } catch { case t: Throwable => throw new RuntimeException("Error in processing replica %d in partition %s at offset %d." - .format(replicaId, topicAndPartition, fetchOffsetMap.get(topicAndPartition)), t) + .format(replicaId, topicPartition, fetchOffsetMap.get(topicPartition)), t) } } if (isMessageInAllReplicas) { val nextOffset = messageInfoFromFirstReplicaOpt.get.nextOffset - fetchOffsetMap.put(topicAndPartition, nextOffset) - debug(expectedReplicasPerTopicAndPartition(topicAndPartition) + " replicas match at offset " + - nextOffset + " for " + topicAndPartition) + fetchOffsetMap.put(topicPartition, nextOffset) + debug(expectedReplicasPerTopicPartition(topicPartition) + " replicas match at offset " + + nextOffset + " for " + topicPartition) } } - if (maxHw - fetchOffsetMap.get(topicAndPartition) > maxLag) { - offsetWithMaxLag = fetchOffsetMap.get(topicAndPartition) + if (maxHw - fetchOffsetMap.get(topicPartition) > maxLag) { + offsetWithMaxLag = fetchOffsetMap.get(topicPartition) maxLag = maxHw - offsetWithMaxLag - maxLagTopicAndPartition = topicAndPartition + maxLagTopicAndPartition = topicPartition } fetchResponsePerReplica.clear() } @@ -334,51 +367,54 @@ 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 + " partitions") + + " among " + recordsCache.size + " partitions") lastReportTime = currentTimeMs } } } -private class ReplicaFetcher(name: String, sourceBroker: BrokerEndPoint, topicAndPartitions: Iterable[TopicAndPartition], +private class ReplicaFetcher(name: String, sourceBroker: Node, topicPartitions: Iterable[TopicPartition], replicaBuffer: ReplicaBuffer, socketTimeout: Int, socketBufferSize: Int, - fetchSize: Int, maxWait: Int, minBytes: Int, doVerification: Boolean) + fetchSize: Int, maxWait: Int, minBytes: Int, doVerification: Boolean, consumerConfig: Properties, + fetcherId: Int) extends ShutdownableThread(name) { - val simpleConsumer = new SimpleConsumer(sourceBroker.host, sourceBroker.port, socketTimeout, socketBufferSize, ReplicaVerificationTool.clientId) - val fetchRequestBuilder = new FetchRequestBuilder(). - clientId(ReplicaVerificationTool.clientId). - replicaId(Request.DebuggingConsumerId). - maxWait(maxWait). - minBytes(minBytes) + + private val fetchEndpoint = new ReplicaFetcherBlockingSend(sourceBroker, new ConsumerConfig(consumerConfig), new Metrics(), Time.SYSTEM, fetcherId, + s"broker-${Request.DebuggingConsumerId}-fetcher-$fetcherId") override def doWork() { val fetcherBarrier = replicaBuffer.getFetcherBarrier() val verificationBarrier = replicaBuffer.getVerificationBarrier() - for (topicAndPartition <- topicAndPartitions) - fetchRequestBuilder.addFetch(topicAndPartition.topic, topicAndPartition.partition, - replicaBuffer.getOffset(topicAndPartition), fetchSize) + val requestMap = new util.LinkedHashMap[TopicPartition, JFetchRequest.PartitionData] + for (topicPartition <- topicPartitions) + requestMap.put(topicPartition, new JFetchRequest.PartitionData(replicaBuffer.getOffset(topicPartition), 0L, fetchSize)) + + val fetchRequestBuilder = JFetchRequest.Builder. + forReplica(ApiKeys.FETCH.latestVersion, Request.DebuggingConsumerId, maxWait, minBytes, requestMap) - val fetchRequest = fetchRequestBuilder.build() - debug("Issuing fetch request " + fetchRequest) + debug("Issuing fetch request ") - var response: FetchResponse = null + var fetchResponse: FetchResponse[MemoryRecords] = null try { - response = simpleConsumer.fetch(fetchRequest) + val clientResponse = fetchEndpoint.sendRequest(fetchRequestBuilder) + fetchResponse = clientResponse.responseBody.asInstanceOf[FetchResponse[MemoryRecords]] } catch { case t: Throwable => if (!isRunning) throw t } - if (response != null) { - response.data.foreach { case (topicAndPartition, partitionData) => - replicaBuffer.addFetchedData(topicAndPartition, sourceBroker.id, partitionData) + if (fetchResponse != null) { + fetchResponse.responseData.asScala.foreach { case (tp, partitionData) => + replicaBuffer.addFetchedData(tp, sourceBroker.id, partitionData) } } else { - for (topicAndPartition <- topicAndPartitions) - replicaBuffer.addFetchedData(topicAndPartition, sourceBroker.id, new FetchResponsePartitionData(messages = MessageSet.Empty)) + val emptyResponse = new FetchResponse.PartitionData(Errors.NONE, FetchResponse.INVALID_HIGHWATERMARK, + FetchResponse.INVALID_LAST_STABLE_OFFSET, FetchResponse.INVALID_LOG_START_OFFSET, null, MemoryRecords.EMPTY) + for (topicAndPartition <- topicPartitions) + replicaBuffer.addFetchedData(topicAndPartition, sourceBroker.id, emptyResponse) } fetcherBarrier.countDown() @@ -402,3 +438,64 @@ private class ReplicaFetcher(name: String, sourceBroker: BrokerEndPoint, topicAn debug("Done verification") } } + +private class ReplicaFetcherBlockingSend(sourceNode: Node, + consumerConfig: ConsumerConfig, + metrics: Metrics, + time: Time, + fetcherId: Int, + clientId: String) { + + private val socketTimeout: Int = consumerConfig.getInt(ConsumerConfig.REQUEST_TIMEOUT_MS_CONFIG) + + private val networkClient = { + val channelBuilder = org.apache.kafka.clients.ClientUtils.createChannelBuilder(consumerConfig) + val selector = new Selector( + NetworkReceive.UNLIMITED, + consumerConfig.getLong(ConsumerConfig.CONNECTIONS_MAX_IDLE_MS_CONFIG), + metrics, + time, + "replica-fetcher", + Map("broker-id" -> sourceNode.id.toString, "fetcher-id" -> fetcherId.toString).asJava, + false, + channelBuilder, + new LogContext + ) + new NetworkClient( + selector, + new ManualMetadataUpdater(), + clientId, + 1, + 0, + 0, + Selectable.USE_DEFAULT_BUFFER_SIZE, + consumerConfig.getInt(ConsumerConfig.RECEIVE_BUFFER_CONFIG), + consumerConfig.getInt(ConsumerConfig.REQUEST_TIMEOUT_MS_CONFIG), + time, + false, + new ApiVersions, + new LogContext + ) + } + + def sendRequest(requestBuilder: Builder[_ <: AbstractRequest]): ClientResponse = { + try { + if (!NetworkClientUtils.awaitReady(networkClient, sourceNode, time, socketTimeout)) + throw new SocketTimeoutException(s"Failed to connect within $socketTimeout ms") + else { + val clientRequest = networkClient.newClientRequest(sourceNode.id.toString, requestBuilder, + time.milliseconds(), true) + NetworkClientUtils.sendAndReceive(networkClient, clientRequest, time) + } + } + catch { + case e: Throwable => + networkClient.close(sourceNode.id.toString) + throw e + } + } + + def close(): Unit = { + networkClient.close() + } +} diff --git a/core/src/main/scala/kafka/tools/SimpleConsumerPerformance.scala b/core/src/main/scala/kafka/tools/SimpleConsumerPerformance.scala deleted file mode 100644 index 888d462f6354f..0000000000000 --- a/core/src/main/scala/kafka/tools/SimpleConsumerPerformance.scala +++ /dev/null @@ -1,165 +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 java.net.URI -import java.text.SimpleDateFormat - -import com.typesafe.scalalogging.LazyLogging -import kafka.api.{FetchRequestBuilder, OffsetRequest, PartitionOffsetRequestInfo} -import kafka.consumer.SimpleConsumer -import kafka.utils._ -import kafka.common.TopicAndPartition -import org.apache.kafka.common.utils.Time - - -/** - * Performance test for the simple consumer - */ -@deprecated("This class has been deprecated and will be removed in a future release.", "0.11.0.0") -object SimpleConsumerPerformance extends LazyLogging { - - def main(args: Array[String]) { - logger.warn("WARNING: SimpleConsumerPerformance is deprecated and will be dropped in a future release following 0.11.0.0.") - - val config = new ConsumerPerfConfig(args) - logger.info("Starting SimpleConsumer...") - - if(!config.hideHeader) { - if(!config.showDetailedStats) - println("start.time, end.time, fetch.size, data.consumed.in.MB, MB.sec, data.consumed.in.nMsg, nMsg.sec") - else - println("time, fetch.size, data.consumed.in.MB, MB.sec, data.consumed.in.nMsg, nMsg.sec") - } - - val consumer = new SimpleConsumer(config.url.getHost, config.url.getPort, 30*1000, 2*config.fetchSize, config.clientId) - - // reset to latest or smallest offset - val topicAndPartition = TopicAndPartition(config.topic, config.partition) - val request = OffsetRequest(Map( - topicAndPartition -> PartitionOffsetRequestInfo(if (config.fromLatest) OffsetRequest.LatestTime else OffsetRequest.EarliestTime, 1) - )) - var offset: Long = consumer.getOffsetsBefore(request).partitionErrorAndOffsets(topicAndPartition).offsets.head - - val startMs = System.currentTimeMillis - var done = false - var totalBytesRead = 0L - var totalMessagesRead = 0L - var consumedInterval = 0 - var lastReportTime: Long = startMs - var lastBytesRead = 0L - var lastMessagesRead = 0L - while(!done) { - // TODO: add in the maxWait and minBytes for performance - val request = new FetchRequestBuilder() - .clientId(config.clientId) - .addFetch(config.topic, config.partition, offset, config.fetchSize) - .build() - val fetchResponse = consumer.fetch(request) - - var messagesRead = 0 - var bytesRead = 0 - val messageSet = fetchResponse.messageSet(config.topic, config.partition) - for (message <- messageSet) { - messagesRead += 1 - bytesRead += message.message.payloadSize - } - - if(messagesRead == 0 || totalMessagesRead > config.numMessages) - done = true - else - // we only did one fetch so we find the offset for the first (head) messageset - offset = messageSet.last.nextOffset - - totalBytesRead += bytesRead - totalMessagesRead += messagesRead - consumedInterval += messagesRead - - if(consumedInterval > config.reportingInterval) { - if(config.showDetailedStats) { - val reportTime = System.currentTimeMillis - val elapsed = (reportTime - lastReportTime)/1000.0 - val totalMBRead = ((totalBytesRead-lastBytesRead)*1.0)/(1024*1024) - println("%s, %d, %.4f, %.4f, %d, %.4f".format(config.dateFormat.format(reportTime), config.fetchSize, - (totalBytesRead*1.0)/(1024*1024), totalMBRead/elapsed, - totalMessagesRead, (totalMessagesRead-lastMessagesRead)/elapsed)) - } - lastReportTime = Time.SYSTEM.milliseconds - lastBytesRead = totalBytesRead - lastMessagesRead = totalMessagesRead - consumedInterval = 0 - } - } - val reportTime = System.currentTimeMillis - val elapsed = (reportTime - startMs) / 1000.0 - - if(!config.showDetailedStats) { - val totalMBRead = (totalBytesRead*1.0)/(1024*1024) - println("%s, %s, %d, %.4f, %.4f, %d, %.4f".format(config.dateFormat.format(startMs), - config.dateFormat.format(reportTime), config.fetchSize, totalMBRead, totalMBRead/elapsed, - totalMessagesRead, totalMessagesRead/elapsed)) - } - Exit.exit(0) - } - - class ConsumerPerfConfig(args: Array[String]) extends PerfConfig(args) { - val urlOpt = parser.accepts("server", "REQUIRED: The hostname of the server to connect to.") - .withRequiredArg - .describedAs("kafka://hostname:port") - .ofType(classOf[String]) - val topicOpt = parser.accepts("topic", "REQUIRED: The topic to consume from.") - .withRequiredArg - .describedAs("topic") - .ofType(classOf[String]) - val resetBeginningOffsetOpt = parser.accepts("from-latest", "If the consumer does not already have an established " + - "offset to consume from, start with the latest message present in the log rather than the earliest message.") - val partitionOpt = parser.accepts("partition", "The topic partition to consume from.") - .withRequiredArg - .describedAs("partition") - .ofType(classOf[java.lang.Integer]) - .defaultsTo(0) - val fetchSizeOpt = parser.accepts("fetch-size", "REQUIRED: The fetch size to use for consumption.") - .withRequiredArg - .describedAs("bytes") - .ofType(classOf[java.lang.Integer]) - .defaultsTo(1024*1024) - val clientIdOpt = parser.accepts("clientId", "The ID of this client.") - .withRequiredArg - .describedAs("clientId") - .ofType(classOf[String]) - .defaultsTo("SimpleConsumerPerformanceClient") - val showDetailedStatsOpt = parser.accepts("show-detailed-stats", "If set, stats are reported for each reporting " + - "interval as configured by reporting-interval") - - val options = parser.parse(args : _*) - - CommandLineUtils.checkRequiredArgs(parser, options, topicOpt, urlOpt, numMessagesOpt) - - val url = new URI(options.valueOf(urlOpt)) - val fetchSize = options.valueOf(fetchSizeOpt).intValue - val fromLatest = options.has(resetBeginningOffsetOpt) - val partition = options.valueOf(partitionOpt).intValue - val topic = options.valueOf(topicOpt) - val numMessages = options.valueOf(numMessagesOpt).longValue - val reportingInterval = options.valueOf(reportingIntervalOpt).intValue - val showDetailedStats = options.has(showDetailedStatsOpt) - val dateFormat = new SimpleDateFormat(options.valueOf(dateFormatOpt)) - val hideHeader = options.has(hideHeaderOpt) - val clientId = options.valueOf(clientIdOpt).toString - } -} diff --git a/core/src/main/scala/kafka/tools/SimpleConsumerShell.scala b/core/src/main/scala/kafka/tools/SimpleConsumerShell.scala deleted file mode 100755 index da8b698f13895..0000000000000 --- a/core/src/main/scala/kafka/tools/SimpleConsumerShell.scala +++ /dev/null @@ -1,263 +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 joptsimple._ -import kafka.utils._ -import kafka.consumer._ -import kafka.client.ClientUtils -import kafka.api.{FetchRequestBuilder, OffsetRequest, Request} -import kafka.cluster.BrokerEndPoint - -import scala.collection.JavaConverters._ -import kafka.common.{MessageFormatter, TopicAndPartition} -import org.apache.kafka.clients.consumer.ConsumerRecord -import org.apache.kafka.common.utils.{KafkaThread, Utils} - -/** - * Command line program to dump out messages to standard out using the simple consumer - */ -@deprecated("This class has been deprecated and will be removed in a future release.", "0.11.0.0") -object SimpleConsumerShell extends Logging { - - def UseLeaderReplica = -1 - - def main(args: Array[String]): Unit = { - warn("WARNING: SimpleConsumerShell is deprecated and will be dropped in a future release following 0.11.0.0.") - - val parser = new OptionParser(false) - val brokerListOpt = parser.accepts("broker-list", "REQUIRED: The list of hostname and port of the server to connect to.") - .withRequiredArg - .describedAs("hostname:port,...,hostname:port") - .ofType(classOf[String]) - val topicOpt = parser.accepts("topic", "REQUIRED: The topic to consume from.") - .withRequiredArg - .describedAs("topic") - .ofType(classOf[String]) - val partitionIdOpt = parser.accepts("partition", "The partition to consume from.") - .withRequiredArg - .describedAs("partition") - .ofType(classOf[java.lang.Integer]) - .defaultsTo(0) - val replicaIdOpt = parser.accepts("replica", "The replica id to consume from, default -1 means leader broker.") - .withRequiredArg - .describedAs("replica id") - .ofType(classOf[java.lang.Integer]) - .defaultsTo(UseLeaderReplica) - val offsetOpt = parser.accepts("offset", "The offset id to consume from, default to -2 which means from beginning; while value -1 means from end") - .withRequiredArg - .describedAs("consume offset") - .ofType(classOf[java.lang.Long]) - .defaultsTo(OffsetRequest.EarliestTime) - val clientIdOpt = parser.accepts("clientId", "The ID of this client.") - .withRequiredArg - .describedAs("clientId") - .ofType(classOf[String]) - .defaultsTo("SimpleConsumerShell") - val fetchSizeOpt = parser.accepts("fetchsize", "The fetch size of each request.") - .withRequiredArg - .describedAs("fetchsize") - .ofType(classOf[java.lang.Integer]) - .defaultsTo(1024 * 1024) - 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) - val messageFormatterArgOpt = parser.accepts("property") - .withRequiredArg - .describedAs("prop") - .ofType(classOf[String]) - val printOffsetOpt = parser.accepts("print-offsets", "Print the offsets returned by the iterator") - val maxWaitMsOpt = parser.accepts("max-wait-ms", "The max amount of time each fetch request waits.") - .withRequiredArg - .describedAs("ms") - .ofType(classOf[java.lang.Integer]) - .defaultsTo(1000) - val maxMessagesOpt = parser.accepts("max-messages", "The number of messages to consume") - .withRequiredArg - .describedAs("max-messages") - .ofType(classOf[java.lang.Integer]) - .defaultsTo(Integer.MAX_VALUE) - val skipMessageOnErrorOpt = parser.accepts("skip-message-on-error", "If there is an error when processing a message, " + - "skip it instead of halt.") - val noWaitAtEndOfLogOpt = parser.accepts("no-wait-at-logend", - "If set, when the simple consumer reaches the end of the Log, it will stop, not waiting for new produced messages") - - if(args.length == 0) - CommandLineUtils.printUsageAndDie(parser, "A low-level tool for fetching data directly from a particular replica.") - - val options = parser.parse(args : _*) - CommandLineUtils.checkRequiredArgs(parser, options, brokerListOpt, topicOpt) - - val topic = options.valueOf(topicOpt) - val partitionId = options.valueOf(partitionIdOpt).intValue() - val replicaId = options.valueOf(replicaIdOpt).intValue() - var startingOffset = options.valueOf(offsetOpt).longValue - val fetchSize = options.valueOf(fetchSizeOpt).intValue - val clientId = options.valueOf(clientIdOpt).toString - val maxWaitMs = options.valueOf(maxWaitMsOpt).intValue() - val maxMessages = options.valueOf(maxMessagesOpt).intValue - - val skipMessageOnError = options.has(skipMessageOnErrorOpt) - val printOffsets = options.has(printOffsetOpt) - val noWaitAtEndOfLog = options.has(noWaitAtEndOfLogOpt) - - val messageFormatterClass = Class.forName(options.valueOf(messageFormatterOpt)) - val formatterArgs = CommandLineUtils.parseKeyValueArgs(options.valuesOf(messageFormatterArgOpt).asScala) - - val fetchRequestBuilder = new FetchRequestBuilder() - .clientId(clientId) - .replicaId(Request.DebuggingConsumerId) - .maxWait(maxWaitMs) - .minBytes(ConsumerConfig.MinFetchBytes) - - // getting topic metadata - info("Getting topic metadata...") - val brokerList = options.valueOf(brokerListOpt) - ToolsUtils.validatePortOrDie(parser,brokerList) - val metadataTargetBrokers = ClientUtils.parseBrokerList(brokerList) - val topicsMetadata = ClientUtils.fetchTopicMetadata(Set(topic), metadataTargetBrokers, clientId, maxWaitMs).topicsMetadata - if(topicsMetadata.size != 1 || !topicsMetadata.head.topic.equals(topic)) { - System.err.println(("Error: no valid topic metadata for topic: %s, " + "what we get from server is only: %s").format(topic, topicsMetadata)) - Exit.exit(1) - } - - // validating partition id - val partitionsMetadata = topicsMetadata.head.partitionsMetadata - val partitionMetadataOpt = partitionsMetadata.find(p => p.partitionId == partitionId) - if (partitionMetadataOpt.isEmpty) { - System.err.println("Error: partition %d does not exist for topic %s".format(partitionId, topic)) - Exit.exit(1) - } - - // validating replica id and initializing target broker - var fetchTargetBroker: BrokerEndPoint = null - var replicaOpt: Option[BrokerEndPoint] = null - if (replicaId == UseLeaderReplica) { - replicaOpt = partitionMetadataOpt.get.leader - if (replicaOpt.isEmpty) { - System.err.println("Error: user specifies to fetch from leader for partition (%s, %d) which has not been elected yet".format(topic, partitionId)) - Exit.exit(1) - } - } - else { - val replicasForPartition = partitionMetadataOpt.get.replicas - replicaOpt = replicasForPartition.find(r => r.id == replicaId) - if(replicaOpt.isEmpty) { - System.err.println("Error: replica %d does not exist for partition (%s, %d)".format(replicaId, topic, partitionId)) - Exit.exit(1) - } - } - fetchTargetBroker = replicaOpt.get - - // initializing starting offset - if(startingOffset < OffsetRequest.EarliestTime) { - System.err.println("Invalid starting offset: %d".format(startingOffset)) - Exit.exit(1) - } - if (startingOffset < 0) { - val simpleConsumer = new SimpleConsumer(fetchTargetBroker.host, - fetchTargetBroker.port, - ConsumerConfig.SocketTimeout, - ConsumerConfig.SocketBufferSize, clientId) - try { - startingOffset = simpleConsumer.earliestOrLatestOffset(TopicAndPartition(topic, partitionId), startingOffset, - Request.DebuggingConsumerId) - } catch { - case t: Throwable => - System.err.println("Error in getting earliest or latest offset due to: " + Utils.stackTrace(t)) - Exit.exit(1) - } finally { - if (simpleConsumer != null) - simpleConsumer.close() - } - } - - // initializing formatter - val formatter = messageFormatterClass.newInstance().asInstanceOf[MessageFormatter] - formatter.init(formatterArgs) - - 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) - val thread = KafkaThread.nonDaemon("kafka-simpleconsumer-shell", new Runnable() { - def run() { - var offset = startingOffset - var numMessagesConsumed = 0 - try { - while (numMessagesConsumed < maxMessages) { - val fetchRequest = fetchRequestBuilder - .addFetch(topic, partitionId, offset, fetchSize) - .build() - val fetchResponse = simpleConsumer.fetch(fetchRequest) - val messageSet = fetchResponse.messageSet(topic, partitionId) - if (messageSet.validBytes <= 0 && noWaitAtEndOfLog) { - println("Terminating. Reached the end of partition (%s, %d) at offset %d".format(topic, partitionId, offset)) - return - } - debug("multi fetched " + messageSet.sizeInBytes + " bytes from offset " + offset) - for (messageAndOffset <- messageSet if numMessagesConsumed < maxMessages) { - try { - offset = messageAndOffset.nextOffset - if (printOffsets) - System.out.println("next offset = " + offset) - val message = messageAndOffset.message - val key = if (message.hasKey) Utils.readBytes(message.key) else null - val value = if (message.isNull) null else Utils.readBytes(message.payload) - val serializedKeySize = if (message.hasKey) key.size else -1 - val serializedValueSize = if (message.isNull) -1 else value.size - formatter.writeTo(new ConsumerRecord(topic, partitionId, offset, message.timestamp, - message.timestampType, message.checksum, serializedKeySize, serializedValueSize, key, value), System.out) - numMessagesConsumed += 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.") - formatter.close() - simpleConsumer.close() - Exit.exit(1) - } - } - } - } catch { - case e: Throwable => - error("Error consuming topic, partition, replica (%s, %d, %d) with offset [%d]".format(topic, partitionId, replicaId, offset), e) - } finally { - info(s"Consumed $numMessagesConsumed messages") - } - } - }) - thread.start() - thread.join() - System.out.flush() - formatter.close() - simpleConsumer.close() - } -} diff --git a/core/src/main/scala/kafka/tools/UpdateOffsetsInZK.scala b/core/src/main/scala/kafka/tools/UpdateOffsetsInZK.scala deleted file mode 100755 index 204f5c18718f0..0000000000000 --- a/core/src/main/scala/kafka/tools/UpdateOffsetsInZK.scala +++ /dev/null @@ -1,93 +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 kafka.consumer.{ConsumerConfig, SimpleConsumer} -import kafka.api.{OffsetRequest, PartitionOffsetRequestInfo} -import kafka.common.{KafkaException, TopicAndPartition} -import kafka.utils.{Exit, Logging, ZKGroupTopicDirs, ZkUtils} -import org.apache.kafka.common.network.ListenerName -import org.apache.kafka.common.security.JaasUtils -import org.apache.kafka.common.security.auth.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. - */ -@deprecated("This class has been deprecated and will be removed in a future release.", "0.11.0.0") -object UpdateOffsetsInZK extends Logging { - val Earliest = "earliest" - val Latest = "latest" - - def main(args: Array[String]) { - warn("WARNING: UpdateOffsetsInZK is deprecated and will be dropped in releases following 0.11.0.0.") - - if(args.length < 3) - usage - val config = new ConsumerConfig(Utils.loadProps(args(1))) - val zkUtils = ZkUtils(config.zkConnect, config.zkSessionTimeoutMs, - config.zkConnectionTimeoutMs, JaasUtils.isZkSecurityEnabled()) - args(0) match { - case Earliest => getAndSetOffsets(zkUtils, OffsetRequest.EarliestTime, config, args(2)) - case Latest => getAndSetOffsets(zkUtils, OffsetRequest.LatestTime, config, args(2)) - case _ => usage - } - } - - private def getAndSetOffsets(zkUtils: ZkUtils, offsetOption: Long, config: ConsumerConfig, topic: String): Unit = { - val partitionsPerTopicMap = zkUtils.getPartitionsForTopics(List(topic)) - var partitions: Seq[Int] = Nil - - partitionsPerTopicMap.get(topic) match { - case Some(l) => partitions = l.sortWith((s,t) => s < t) - case _ => throw new RuntimeException("Can't find topic " + topic) - } - - var numParts = 0 - for (partition <- partitions) { - val brokerHostingPartition = zkUtils.getLeaderForPartition(topic, partition) - - val broker = brokerHostingPartition match { - case Some(b) => b - case None => throw new KafkaException("Broker " + brokerHostingPartition + " is unavailable. Cannot issue " + - "getOffsetsBefore request") - } - - zkUtils.getBrokerInfo(broker) match { - case Some(brokerInfo) => - val brokerEndPoint = brokerInfo.brokerEndPoint(ListenerName.forSecurityProtocol(SecurityProtocol.PLAINTEXT)) - val consumer = new SimpleConsumer(brokerEndPoint.host, brokerEndPoint.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 - val topicDirs = new ZKGroupTopicDirs(config.groupId, topic) - - println("updating partition " + partition + " with new offset: " + offset) - zkUtils.updatePersistentPath(topicDirs.consumerOffsetDir + "/" + partition, offset.toString) - numParts += 1 - case None => throw new KafkaException("Broker information for broker id %d does not exist in ZK".format(broker)) - } - } - println("updated the offset for " + numParts + " partitions") - } - - private def usage() = { - println("USAGE: " + UpdateOffsetsInZK.getClass.getName + " [earliest | latest] consumer.properties topic") - Exit.exit(1) - } -} diff --git a/core/src/main/scala/kafka/tools/VerifyConsumerRebalance.scala b/core/src/main/scala/kafka/tools/VerifyConsumerRebalance.scala deleted file mode 100644 index 164595714d20e..0000000000000 --- a/core/src/main/scala/kafka/tools/VerifyConsumerRebalance.scala +++ /dev/null @@ -1,139 +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 joptsimple.OptionParser -import org.apache.kafka.common.security._ -import kafka.utils.{CommandLineUtils, Exit, Logging, ZKGroupTopicDirs, ZkUtils} - -@deprecated("This class has been deprecated and will be removed in a future release.", "0.11.0.0") -object VerifyConsumerRebalance extends Logging { - def main(args: Array[String]) { - val parser = new OptionParser(false) - warn("WARNING: VerifyConsumerRebalance is deprecated and will be dropped in a future release following 0.11.0.0.") - - val zkConnectOpt = parser.accepts("zookeeper.connect", "ZooKeeper connect string."). - withRequiredArg().defaultsTo("localhost:2181").ofType(classOf[String]) - val groupOpt = parser.accepts("group", "Consumer group."). - withRequiredArg().ofType(classOf[String]) - parser.accepts("help", "Print this message.") - - if(args.length == 0) - CommandLineUtils.printUsageAndDie(parser, "Validate that all partitions have a consumer for a given consumer group.") - - val options = parser.parse(args : _*) - - if (options.has("help")) { - parser.printHelpOn(System.out) - Exit.exit(0) - } - - CommandLineUtils.checkRequiredArgs(parser, options, groupOpt) - - val zkConnect = options.valueOf(zkConnectOpt) - val group = options.valueOf(groupOpt) - - var zkUtils: ZkUtils = null - try { - zkUtils = ZkUtils(zkConnect, - 30000, - 30000, - JaasUtils.isZkSecurityEnabled()) - - debug("zkConnect = %s; group = %s".format(zkConnect, group)) - - // check if the rebalancing operation succeeded. - try { - if(validateRebalancingOperation(zkUtils, group)) - println("Rebalance operation successful !") - else - println("Rebalance operation failed !") - } catch { - case e2: Throwable => error("Error while verifying current rebalancing operation", e2) - } - } - finally { - if (zkUtils != null) - zkUtils.close() - } - } - - private def validateRebalancingOperation(zkUtils: ZkUtils, group: String): Boolean = { - info("Verifying rebalancing operation for consumer group " + group) - var rebalanceSucceeded: Boolean = true - /** - * A successful rebalancing operation would select an owner for each available partition - * This means that for each partition registered under /brokers/topics/[topic]/[broker-id], an owner exists - * under /consumers/[consumer_group]/owners/[topic]/[broker_id-partition_id] - */ - val consumersPerTopicMap = zkUtils.getConsumersPerTopic(group, excludeInternalTopics = false) - val partitionsPerTopicMap = zkUtils.getPartitionsForTopics(consumersPerTopicMap.keySet.toSeq) - - 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))) - val partitionsWithOwners = zkUtils.getChildrenParentMayNotExist(topicDirs.consumerOwnerDir) - if(partitionsWithOwners.isEmpty) { - error("No owners for any partitions for topic " + topic) - rebalanceSucceeded = false - } - debug("Children of " + topicDirs.consumerOwnerDir + " = " + partitionsWithOwners.toString) - val consumerIdsForTopic = consumersPerTopicMap.get(topic) - - // for each available partition for topic, check if an owner exists - partitions.foreach { 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 - } - // try reading the partition owner path for see if a valid consumer id exists there - val partitionOwnerPath = topicDirs.consumerOwnerDir + "/" + partition - val partitionOwner = zkUtils.readDataMaybeNull(partitionOwnerPath)._1 match { - case Some(m) => m - case None => null - } - if(partitionOwner == null) { - error("No owner for partition [%s,%d]".format(topic, partition)) - rebalanceSucceeded = false - } - else { - // check if the owner is a valid consumer id - consumerIdsForTopic match { - case Some(consumerIds) => - if(!consumerIds.map(c => c.toString).contains(partitionOwner)) { - error(("Owner %s for partition [%s,%d] is not a valid member of consumer " + - "group %s").format(partitionOwner, topic, partition, group)) - rebalanceSucceeded = false - } - else - info("Owner of partition [%s,%d] is %s".format(topic, partition, partitionOwner)) - case None => { - error("No consumer ids registered for topic " + topic) - rebalanceSucceeded = false - } - } - } - } - - } - - rebalanceSucceeded - } -} diff --git a/core/src/main/scala/kafka/utils/Pool.scala b/core/src/main/scala/kafka/utils/Pool.scala index 4ddf55736dde8..5dd61946f0d29 100644 --- a/core/src/main/scala/kafka/utils/Pool.scala +++ b/core/src/main/scala/kafka/utils/Pool.scala @@ -19,14 +19,14 @@ package kafka.utils import java.util.concurrent._ +import org.apache.kafka.common.KafkaException + import collection.mutable import collection.JavaConverters._ -import kafka.common.KafkaException class Pool[K,V](valueFactory: Option[K => V] = None) extends Iterable[(K, V)] { private val pool: ConcurrentMap[K, V] = new ConcurrentHashMap[K, V] - private val createLock = new Object def put(k: K, v: V): V = pool.put(k, v) @@ -56,21 +56,10 @@ class Pool[K,V](valueFactory: Option[K => V] = None) extends Iterable[(K, V)] { * @param createValue Factory function. * @return The final value associated with the key. */ - def getAndMaybePut(key: K, createValue: => V): V = { - val current = pool.get(key) - if (current == null) { - createLock synchronized { - val current = pool.get(key) - if (current == null) { - val value = createValue - pool.put(key, value) - value - } - else current - } - } - else current - } + def getAndMaybePut(key: K, createValue: => V): V = + pool.computeIfAbsent(key, new java.util.function.Function[K, V] { + override def apply(k: K): V = createValue + }) def contains(id: K): Boolean = pool.containsKey(id) diff --git a/core/src/main/scala/kafka/consumer/TopicFilter.scala b/core/src/main/scala/kafka/utils/TopicFilter.scala similarity index 72% rename from core/src/main/scala/kafka/consumer/TopicFilter.scala rename to core/src/main/scala/kafka/utils/TopicFilter.scala index b71b01acd5ed7..64e7d2acea830 100644 --- a/core/src/main/scala/kafka/consumer/TopicFilter.scala +++ b/core/src/main/scala/kafka/utils/TopicFilter.scala @@ -15,14 +15,12 @@ * limitations under the License. */ -package kafka.consumer +package kafka.utils -import kafka.utils.Logging import java.util.regex.{Pattern, PatternSyntaxException} import org.apache.kafka.common.internals.Topic -@deprecated("This class has been deprecated and will be removed in a future release.", "0.11.0.0") sealed abstract class TopicFilter(rawRegex: String) extends Logging { val regex = rawRegex @@ -45,7 +43,6 @@ sealed abstract class TopicFilter(rawRegex: String) extends Logging { def isTopicAllowed(topic: String, excludeInternalTopics: Boolean): Boolean } -@deprecated("This class has been deprecated and will be removed in a future release.", "0.11.0.0") case class Whitelist(rawRegex: String) extends TopicFilter(rawRegex) { override def isTopicAllowed(topic: String, excludeInternalTopics: Boolean) = { val allowed = topic.matches(regex) && !(Topic.isInternal(topic) && excludeInternalTopics) @@ -56,16 +53,3 @@ case class Whitelist(rawRegex: String) extends TopicFilter(rawRegex) { allowed } } - -@deprecated("This class has been deprecated and will be removed in a future release.", "0.11.0.0") -case class Blacklist(rawRegex: String) extends TopicFilter(rawRegex) { - override def isTopicAllowed(topic: String, excludeInternalTopics: Boolean) = { - val allowed = (!topic.matches(regex)) && !(Topic.isInternal(topic) && excludeInternalTopics) - - debug("%s %s".format( - topic, if (allowed) "allowed" else "filtered")) - - allowed - } -} - diff --git a/core/src/main/scala/kafka/utils/ZkUtils.scala b/core/src/main/scala/kafka/utils/ZkUtils.scala index 01799370ef8bd..d47af0d51c227 100644 --- a/core/src/main/scala/kafka/utils/ZkUtils.scala +++ b/core/src/main/scala/kafka/utils/ZkUtils.scala @@ -18,28 +18,26 @@ package kafka.utils import java.nio.charset.StandardCharsets -import java.util.concurrent.CountDownLatch import kafka.admin._ -import kafka.api.{ApiVersion, KAFKA_0_10_0_IV1, LeaderAndIsr} +import kafka.api.LeaderAndIsr import kafka.cluster._ import kafka.common.{KafkaException, NoEpochForPartitionException, TopicAndPartition} -import kafka.consumer.{ConsumerThreadId, TopicCount} import kafka.controller.{LeaderIsrAndControllerEpoch, ReassignedPartitionsContext} import kafka.zk.{BrokerIdZNode, ReassignPartitionsZNode, ZkData} -import org.I0Itec.zkclient.exception.{ZkBadVersionException, ZkException, ZkMarshallingError, ZkNoNodeException, ZkNodeExistsException} +import org.I0Itec.zkclient.exception.{ZkBadVersionException, ZkMarshallingError, ZkNoNodeException, ZkNodeExistsException} import org.I0Itec.zkclient.serialize.ZkSerializer import org.I0Itec.zkclient.{IZkChildListener, IZkDataListener, IZkStateListener, ZkClient, ZkConnection} import org.apache.kafka.common.config.ConfigException -import org.apache.zookeeper.AsyncCallback.{DataCallback, StringCallback} -import org.apache.zookeeper.KeeperException.Code import org.apache.zookeeper.data.{ACL, Stat} -import org.apache.zookeeper.{CreateMode, KeeperException, ZooDefs, ZooKeeper} +import org.apache.zookeeper.ZooDefs import scala.collection._ import scala.collection.JavaConverters._ import org.apache.kafka.common.TopicPartition +@deprecated("This is an internal class that is no longer used by Kafka and will be removed in a future release. Please " + + "use org.apache.kafka.clients.admin.AdminClient instead.", since = "2.0.0") object ZkUtils { private val UseDefaultAcls = new java.util.ArrayList[ACL] @@ -96,9 +94,6 @@ object ZkUtils { (zkClient, zkConnection) } - @deprecated("This is deprecated, use defaultAcls(isSecure, path) which doesn't make sensitive data world readable", since = "0.10.2.1") - def DefaultAcls(isSecure: Boolean): java.util.List[ACL] = defaultAcls(isSecure, "") - def sensitivePath(path: String): Boolean = ZkData.sensitivePath(path) def defaultAcls(isSecure: Boolean, path: String): java.util.List[ACL] = ZkData.defaultAcls(isSecure, path).asJava @@ -190,6 +185,8 @@ object ZkUtils { /** * Legacy class for interacting with ZooKeeper. Whenever possible, ``KafkaZkClient`` should be used instead. */ +@deprecated("This is an internal class that is no longer used by Kafka and will be removed in a future release. Please " + + "use org.apache.kafka.clients.admin.AdminClient instead.", since = "2.0.0") class ZkUtils(val zkClient: ZkClient, val zkConnection: ZkConnection, val isSecure: Boolean) extends Logging { @@ -201,9 +198,6 @@ class ZkUtils(val zkClient: ZkClient, // Visible for testing val zkPath = new ZkPath(zkClient) - @deprecated("This is deprecated, use defaultAcls(path) which doesn't make sensitive data world readable", since = "0.10.2.1") - val DefaultAcls: java.util.List[ACL] = ZkUtils.defaultAcls(isSecure, "") - def defaultAcls(path: String): java.util.List[ACL] = ZkUtils.defaultAcls(isSecure, path) def getController(): Int = { @@ -349,62 +343,6 @@ class ZkUtils(val zkClient: ZkClient, seqOpt.getOrElse(Seq.empty) } - /** - * Register brokers with v4 json format (which includes multiple endpoints and rack) if - * the apiVersion is 0.10.0.X or above. Register the broker with v2 json format otherwise. - * Due to KAFKA-3100, 0.9.0.0 broker and old clients will break if JSON version is above 2. - * We include v2 to make it possible for the broker to migrate from 0.9.0.0 to 0.10.0.X or above without having to - * upgrade to 0.9.0.1 first (clients have to be upgraded to 0.9.0.1 in any case). - * - * This format also includes default endpoints for compatibility with older clients. - * - * @param id broker ID - * @param host broker host name - * @param port broker port - * @param advertisedEndpoints broker end points - * @param jmxPort jmx port - * @param rack broker rack - * @param apiVersion Kafka version the broker is running as - */ - def registerBrokerInZk(id: Int, - host: String, - port: Int, - advertisedEndpoints: Seq[EndPoint], - jmxPort: Int, - rack: Option[String], - apiVersion: ApiVersion) { - val brokerIdPath = BrokerIdsPath + "/" + id - // see method documentation for reason why we do this - val version = if (apiVersion >= KAFKA_0_10_0_IV1) 4 else 2 - val json = new String(BrokerIdZNode.encode(version, host, port, advertisedEndpoints, jmxPort, rack), - StandardCharsets.UTF_8) - registerBrokerInZk(brokerIdPath, json) - - info("Registered broker %d at path %s with addresses: %s".format(id, brokerIdPath, advertisedEndpoints.mkString(","))) - } - - private def registerBrokerInZk(brokerIdPath: String, brokerInfo: String) { - try { - val zkCheckedEphemeral = new ZKCheckedEphemeral(brokerIdPath, - brokerInfo, - zkConnection.getZookeeper, - isSecure) - zkCheckedEphemeral.create() - } catch { - case _: 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.") - } - } - - @deprecated("This method has been deprecated and will be removed in a future release.", "0.11.0.0") - def getConsumerPartitionOwnerPath(group: String, topic: String, partition: Int): String = { - val topicDirs = new ZKGroupTopicDirs(group, topic) - topicDirs.consumerOwnerDir + "/" + partition - } - def leaderAndIsrZkData(leaderAndIsr: LeaderAndIsr, controllerEpoch: Int): String = { Json.legacyEncodeAsString(Map("version" -> 1, "leader" -> leaderAndIsr.leader, "leader_epoch" -> leaderAndIsr.leaderEpoch, "controller_epoch" -> controllerEpoch, "isr" -> leaderAndIsr.isr)) @@ -809,42 +747,6 @@ class ZkUtils(val zkClient: ZkClient, zkClient.delete(brokerPartTopicPath) } - @deprecated("This method has been deprecated and will be removed in a future release.", "0.11.0.0") - def getConsumersInGroup(group: String): Seq[String] = { - val dirs = new ZKGroupDirs(group) - getChildren(dirs.consumerRegistryDir) - } - - @deprecated("This method has been deprecated and will be removed in a future release.", "0.11.0.0") - def getConsumersPerTopic(group: String, excludeInternalTopics: Boolean): mutable.Map[String, List[ConsumerThreadId]] = { - val dirs = new ZKGroupDirs(group) - val consumers = getChildrenParentMayNotExist(dirs.consumerRegistryDir) - val consumersPerTopicMap = new mutable.HashMap[String, List[ConsumerThreadId]] - for (consumer <- consumers) { - val topicCount = TopicCount.constructTopicCount(group, consumer, this, excludeInternalTopics) - for ((topic, consumerThreadIdSet) <- topicCount.getConsumerThreadIdsPerTopic) { - for (consumerThreadId <- consumerThreadIdSet) - consumersPerTopicMap.get(topic) match { - case Some(curConsumers) => consumersPerTopicMap.put(topic, consumerThreadId :: curConsumers) - case _ => consumersPerTopicMap.put(topic, List(consumerThreadId)) - } - } - } - for ((topic, consumerList) <- consumersPerTopicMap) - consumersPerTopicMap.put(topic, consumerList.sortWith((s, t) => s < t)) - consumersPerTopicMap - } - - @deprecated("This method has been deprecated and will be removed in a future release.", "0.11.0.0") - def getTopicsPerMemberId(group: String, excludeInternalTopics: Boolean = true): Map[String, List[String]] = { - val dirs = new ZKGroupDirs(group) - val memberIds = getChildrenParentMayNotExist(dirs.consumerRegistryDir) - memberIds.map { memberId => - val topicCount = TopicCount.constructTopicCount(group, memberId, this, excludeInternalTopics) - memberId -> topicCount.getTopicCountMap.keys.toList - }.toMap - } - /** * This API takes in a broker id, queries zookeeper for the broker metadata and returns the metadata for that broker * or throws an exception if the broker dies before the query to zookeeper finishes @@ -906,29 +808,6 @@ class ZkUtils(val zkClient: ZkClient, } } - @deprecated("This method has been deprecated and will be removed in a future release.", "0.11.0.0") - def getConsumerGroups() = { - getChildren(ConsumersPath) - } - - @deprecated("This method has been deprecated and will be removed in a future release.", "0.11.0.0") - def getTopicsByConsumerGroup(consumerGroup:String) = { - getChildrenParentMayNotExist(new ZKGroupDirs(consumerGroup).consumerGroupOwnersDir) - } - - @deprecated("This method has been deprecated and will be removed in a future release.", "0.11.0.0") - def getAllConsumerGroupsForTopic(topic: String): Set[String] = { - val groups = getChildrenParentMayNotExist(ConsumersPath) - if (groups == null) Set.empty - else { - groups.foldLeft(Set.empty[String]) {(consumerGroupsForTopic, group) => - val topics = getChildren(new ZKGroupDirs(group).consumerGroupOffsetsDir) - if (topics.contains(topic)) consumerGroupsForTopic + group - else consumerGroupsForTopic - } - } - } - def close() { zkClient.close() } @@ -948,21 +827,6 @@ private object ZKStringSerializer extends ZkSerializer { } } -@deprecated("This class has been deprecated and will be removed in a future release.", "0.11.0.0") -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" -} - -@deprecated("This class has been deprecated and will be removed in a future release.", "0.11.0.0") -class ZKGroupTopicDirs(group: String, topic: String) extends ZKGroupDirs(group) { - def consumerOffsetDir = consumerGroupOffsetsDir + "/" + topic - def consumerOwnerDir = consumerGroupOwnersDir + "/" + topic -} - object ZKConfig { val ZkConnectProp = "zookeeper.connect" val ZkSessionTimeoutMsProp = "zookeeper.session.timeout.ms" @@ -1024,169 +888,3 @@ class ZkPath(zkClient: ZkClient) { zkClient.createPersistentSequential(path, data, acls) } } - -/** - * 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, - isSecure: Boolean) extends Logging { - private val createCallback = new CreateCallback - private val getDataCallback = new GetDataCallback - val latch: CountDownLatch = new CountDownLatch(1) - var result: Code = Code.OK - val defaultAcls = ZkUtils.defaultAcls(isSecure, path) - - 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 Code.INVALIDACL => - error("Invalid ACL") - setResult(Code.INVALIDACL) - 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 Code.INVALIDACL => - error("Invalid ACL") - setResult(Code.INVALIDACL) - 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), - defaultAcls, - 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), - defaultAcls, - 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 Code.INVALIDACL => - error("Invalid ACL") - setResult(Code.INVALIDACL) - 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(s"Path: $path, Prefix: $prefix, Suffix: $suffix") - info(s"Creating $path (is it secure? $isSecure)") - 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/zk/AdminZkClient.scala b/core/src/main/scala/kafka/zk/AdminZkClient.scala index 2f8da360c9be9..060c0b4d4aec8 100644 --- a/core/src/main/scala/kafka/zk/AdminZkClient.scala +++ b/core/src/main/scala/kafka/zk/AdminZkClient.scala @@ -93,7 +93,6 @@ class AdminZkClient(zkClient: KafkaZkClient) extends Logging { update: Boolean = false) { validateCreateOrUpdateTopic(topic, partitionReplicaAssignment, config, update) - // 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 zkClient.setOrCreateEntityConfigs(ConfigType.Topic, topic, config) @@ -265,6 +264,18 @@ class AdminZkClient(zkClient: KafkaZkClient) extends Logging { } } + def parseBroker(broker: String): Option[Int] = { + broker match { + case ConfigEntityName.Default => None + case _ => + try Some(broker.toInt) + catch { + case _: NumberFormatException => + throw new IllegalArgumentException(s"Error parsing broker $broker. The broker's Entity Name must be a single integer value") + } + } + } + /** * Change the configs for a given entityType and entityName * @param entityType @@ -273,19 +284,11 @@ class AdminZkClient(zkClient: KafkaZkClient) extends Logging { */ def changeConfigs(entityType: String, entityName: String, configs: Properties): Unit = { - def parseBroker(broker: String): Int = { - try broker.toInt - catch { - case _: NumberFormatException => - throw new IllegalArgumentException(s"Error parsing broker $broker. The broker's Entity Name must be a single integer value") - } - } - entityType match { case ConfigType.Topic => changeTopicConfig(entityName, configs) case ConfigType.Client => changeClientIdConfig(entityName, configs) case ConfigType.User => changeUserOrUserClientIdConfig(entityName, configs) - case ConfigType.Broker => changeBrokerConfig(Seq(parseBroker(entityName)), configs) + case ConfigType.Broker => changeBrokerConfig(parseBroker(entityName), configs) case _ => throw new IllegalArgumentException(s"$entityType is not a known entityType. Should be one of ${ConfigType.Topic}, ${ConfigType.Client}, ${ConfigType.Broker}") } } diff --git a/core/src/main/scala/kafka/zk/KafkaZkClient.scala b/core/src/main/scala/kafka/zk/KafkaZkClient.scala index 90f71a1e1d201..0406457cde4d6 100644 --- a/core/src/main/scala/kafka/zk/KafkaZkClient.scala +++ b/core/src/main/scala/kafka/zk/KafkaZkClient.scala @@ -21,16 +21,16 @@ import java.util.Properties import com.yammer.metrics.core.MetricName import kafka.api.LeaderAndIsr import kafka.cluster.Broker -import kafka.common.KafkaException import kafka.controller.LeaderIsrAndControllerEpoch import kafka.log.LogConfig import kafka.metrics.KafkaMetricsGroup -import kafka.security.auth.SimpleAclAuthorizer.VersionedAcls -import kafka.security.auth.{Acl, Resource, ResourceNameType, ResourceType} +import kafka.security.auth.SimpleAclAuthorizer.{VersionedAcls, NoAcls} +import kafka.security.auth.{Acl, Resource, ResourceType} import kafka.server.ConfigType import kafka.utils.Logging import kafka.zookeeper._ -import org.apache.kafka.common.TopicPartition +import org.apache.kafka.common.{KafkaException, TopicPartition} +import org.apache.kafka.common.resource.PatternType import org.apache.kafka.common.security.token.delegation.{DelegationToken, TokenInformation} import org.apache.kafka.common.utils.{Time, Utils} import org.apache.zookeeper.KeeperException.{Code, NodeExistsException} @@ -87,7 +87,7 @@ class KafkaZkClient private (zooKeeperClient: ZooKeeperClient, isSecure: Boolean def updateBrokerInfoInZk(brokerInfo: BrokerInfo): Unit = { val brokerIdPath = brokerInfo.path - val setDataRequest = SetDataRequest(brokerIdPath, brokerInfo.toJsonBytes, ZkVersion.NoVersion) + val setDataRequest = SetDataRequest(brokerIdPath, brokerInfo.toJsonBytes, ZkVersion.MatchAnyVersion) val response = retryRequestUntilConnected(setDataRequest) response.maybeThrow() info("Updated broker %d at path %s with addresses: %s".format(brokerInfo.broker.id, brokerIdPath, brokerInfo.broker.endPoints)) @@ -246,6 +246,12 @@ class KafkaZkClient private (zooKeeperClient: ZooKeeperClient, isSecure: Boolean /** * Sets or creates the entity znode path with the given configs depending * on whether it already exists or not. + * + * If this is method is called concurrently, the last writer wins. In cases where we update configs and then + * partition assignment (i.e. create topic), it's possible for one thread to set this and the other to set the + * partition assignment. As such, the recommendation is to never call create topic for the same topic with different + * configs/partition assignment concurrently. + * * @param rootEntityType entity type * @param sanitizedEntityName entity name * @throws KeeperException if there is an error while setting or creating the znode @@ -253,20 +259,24 @@ class KafkaZkClient private (zooKeeperClient: ZooKeeperClient, isSecure: Boolean def setOrCreateEntityConfigs(rootEntityType: String, sanitizedEntityName: String, config: Properties) = { def set(configData: Array[Byte]): SetDataResponse = { - val setDataRequest = SetDataRequest(ConfigEntityZNode.path(rootEntityType, sanitizedEntityName), ConfigEntityZNode.encode(config), ZkVersion.NoVersion) + val setDataRequest = SetDataRequest(ConfigEntityZNode.path(rootEntityType, sanitizedEntityName), + ConfigEntityZNode.encode(config), ZkVersion.MatchAnyVersion) retryRequestUntilConnected(setDataRequest) } - def create(configData: Array[Byte]) = { + def createOrSet(configData: Array[Byte]): Unit = { val path = ConfigEntityZNode.path(rootEntityType, sanitizedEntityName) - createRecursive(path, ConfigEntityZNode.encode(config)) + try createRecursive(path, ConfigEntityZNode.encode(config)) + catch { + case _: NodeExistsException => set(configData).maybeThrow + } } val configData = ConfigEntityZNode.encode(config) val setDataResponse = set(configData) setDataResponse.resultCode match { - case Code.NONODE => create(configData) + case Code.NONODE => createOrSet(configData) case _ => setDataResponse.maybeThrow } } @@ -363,7 +373,7 @@ class KafkaZkClient private (zooKeeperClient: ZooKeeperClient, isSecure: Boolean * @return SetDataResponse */ def setTopicAssignmentRaw(topic: String, assignment: collection.Map[TopicPartition, Seq[Int]]): SetDataResponse = { - val setDataRequest = SetDataRequest(TopicZNode.path(topic), TopicZNode.encode(assignment), ZkVersion.NoVersion) + val setDataRequest = SetDataRequest(TopicZNode.path(topic), TopicZNode.encode(assignment), ZkVersion.MatchAnyVersion) retryRequestUntilConnected(setDataRequest) } @@ -438,7 +448,7 @@ class KafkaZkClient private (zooKeeperClient: ZooKeeperClient, isSecure: Boolean */ def deleteLogDirEventNotifications(sequenceNumbers: Seq[String]): Unit = { val deleteRequests = sequenceNumbers.map { sequenceNumber => - DeleteRequest(LogDirEventNotificationSequenceZNode.path(sequenceNumber), ZkVersion.NoVersion) + DeleteRequest(LogDirEventNotificationSequenceZNode.path(sequenceNumber), ZkVersion.MatchAnyVersion) } retryRequestsUntilConnected(deleteRequests) } @@ -538,12 +548,12 @@ class KafkaZkClient private (zooKeeperClient: ZooKeeperClient, isSecure: Boolean * @param path zk node path * @return A tuple of 2 elements, where first element is zk node data as an array of bytes * and second element is zk node version. - * returns (None, ZkVersion.NoVersion) if node doesn't exists and throws exception for any error + * returns (None, ZkVersion.UnknownVersion) if node doesn't exist and throws exception for any error */ def getDataAndVersion(path: String): (Option[Array[Byte]], Int) = { val (data, stat) = getDataAndStat(path) stat match { - case ZkStat.NoStat => (data, ZkVersion.NoVersion) + case ZkStat.NoStat => (data, ZkVersion.UnknownVersion) case _ => (data, stat.getVersion) } } @@ -582,7 +592,7 @@ class KafkaZkClient private (zooKeeperClient: ZooKeeperClient, isSecure: Boolean /** * Conditional update the persistent path data, return (true, newVersion) if it succeeds, otherwise (the path doesn't - * exist, the current version is not the expected version, etc.) return (false, ZkVersion.NoVersion) + * exist, the current version is not the expected version, etc.) return (false, ZkVersion.UnknownVersion) * * When there is a ConnectionLossException during the conditional update, ZookeeperClient will retry the update and may fail * since the previous update may have succeeded (but the stored zkVersion no longer matches the expected one). @@ -607,13 +617,13 @@ class KafkaZkClient private (zooKeeperClient: ZooKeeperClient, isSecure: Boolean debug("Checker method is not passed skipping zkData match") debug("Conditional update of path %s with data %s and expected version %d failed due to %s" .format(path, Utils.utf8(data), expectVersion, setDataResponse.resultException.get.getMessage)) - (false, ZkVersion.NoVersion) + (false, ZkVersion.UnknownVersion) } case Code.NONODE => debug("Conditional update of path %s with data %s and expected version %d failed due to %s".format(path, Utils.utf8(data), expectVersion, setDataResponse.resultException.get.getMessage)) - (false, ZkVersion.NoVersion) + (false, ZkVersion.UnknownVersion) case _ => debug("Conditional update of path %s with data %s and expected version %d failed due to %s".format(path, @@ -658,7 +668,7 @@ class KafkaZkClient private (zooKeeperClient: ZooKeeperClient, isSecure: Boolean * @param topics the topics to remove. */ def deleteTopicDeletions(topics: Seq[String]): Unit = { - val deleteRequests = topics.map(topic => DeleteRequest(DeleteTopicsTopicZNode.path(topic), ZkVersion.NoVersion)) + val deleteRequests = topics.map(topic => DeleteRequest(DeleteTopicsTopicZNode.path(topic), ZkVersion.MatchAnyVersion)) retryRequestsUntilConnected(deleteRequests) } @@ -692,7 +702,7 @@ class KafkaZkClient private (zooKeeperClient: ZooKeeperClient, isSecure: Boolean def setOrCreatePartitionReassignment(reassignment: collection.Map[TopicPartition, Seq[Int]]): Unit = { def set(reassignmentData: Array[Byte]): SetDataResponse = { - val setDataRequest = SetDataRequest(ReassignPartitionsZNode.path, reassignmentData, ZkVersion.NoVersion) + val setDataRequest = SetDataRequest(ReassignPartitionsZNode.path, reassignmentData, ZkVersion.MatchAnyVersion) retryRequestUntilConnected(setDataRequest) } @@ -725,7 +735,7 @@ class KafkaZkClient private (zooKeeperClient: ZooKeeperClient, isSecure: Boolean * Deletes the partition reassignment znode. */ def deletePartitionReassignment(): Unit = { - val deleteRequest = DeleteRequest(ReassignPartitionsZNode.path, ZkVersion.NoVersion) + val deleteRequest = DeleteRequest(ReassignPartitionsZNode.path, ZkVersion.MatchAnyVersion) retryRequestUntilConnected(deleteRequest) } @@ -846,7 +856,7 @@ class KafkaZkClient private (zooKeeperClient: ZooKeeperClient, isSecure: Boolean */ def deleteIsrChangeNotifications(sequenceNumbers: Seq[String]): Unit = { val deleteRequests = sequenceNumbers.map { sequenceNumber => - DeleteRequest(IsrChangeNotificationSequenceZNode.path(sequenceNumber), ZkVersion.NoVersion) + DeleteRequest(IsrChangeNotificationSequenceZNode.path(sequenceNumber), ZkVersion.MatchAnyVersion) } retryRequestsUntilConnected(deleteRequests) } @@ -878,7 +888,7 @@ class KafkaZkClient private (zooKeeperClient: ZooKeeperClient, isSecure: Boolean * Deletes the preferred replica election znode. */ def deletePreferredReplicaElection(): Unit = { - val deleteRequest = DeleteRequest(PreferredReplicaElectionZNode.path, ZkVersion.NoVersion) + val deleteRequest = DeleteRequest(PreferredReplicaElectionZNode.path, ZkVersion.MatchAnyVersion) retryRequestUntilConnected(deleteRequest) } @@ -900,7 +910,7 @@ class KafkaZkClient private (zooKeeperClient: ZooKeeperClient, isSecure: Boolean * Deletes the controller znode. */ def deleteController(): Unit = { - val deleteRequest = DeleteRequest(ControllerZNode.path, ZkVersion.NoVersion) + val deleteRequest = DeleteRequest(ControllerZNode.path, ZkVersion.MatchAnyVersion) retryRequestUntilConnected(deleteRequest) } @@ -933,21 +943,22 @@ class KafkaZkClient private (zooKeeperClient: ZooKeeperClient, isSecure: Boolean * @param topics the topics whose configs we wish to delete. */ def deleteTopicConfigs(topics: Seq[String]): Unit = { - val deleteRequests = topics.map(topic => DeleteRequest(ConfigEntityZNode.path(ConfigType.Topic, topic), ZkVersion.NoVersion)) + val deleteRequests = topics.map(topic => DeleteRequest(ConfigEntityZNode.path(ConfigType.Topic, topic), ZkVersion.MatchAnyVersion)) retryRequestsUntilConnected(deleteRequests) } //Acl management methods /** - * Creates the required zk nodes for Acl storage + * Creates the required zk nodes for Acl storage and Acl change storage. */ def createAclPaths(): Unit = { ZkAclStore.stores.foreach(store => { createRecursive(store.aclPath, throwIfPathExists = false) - createRecursive(store.aclChangePath, throwIfPathExists = false) ResourceType.values.foreach(resourceType => createRecursive(store.path(resourceType), throwIfPathExists = false)) }) + + ZkAclChangeStore.stores.foreach(store => createRecursive(store.aclChangePath, throwIfPathExists = false)) } /** @@ -960,7 +971,7 @@ class KafkaZkClient private (zooKeeperClient: ZooKeeperClient, isSecure: Boolean val getDataResponse = retryRequestUntilConnected(getDataRequest) getDataResponse.resultCode match { case Code.OK => ResourceZNode.decode(getDataResponse.data, getDataResponse.stat) - case Code.NONODE => VersionedAcls(Set(), -1) + case Code.NONODE => NoAcls case _ => throw getDataResponse.resultException.get } } @@ -973,12 +984,26 @@ class KafkaZkClient private (zooKeeperClient: ZooKeeperClient, isSecure: Boolean * @param expectedVersion * @return true if the update was successful and the new version */ - def conditionalSetOrCreateAclsForResource(resource: Resource, aclsSet: Set[Acl], expectedVersion: Int): (Boolean, Int) = { + def conditionalSetAclsForResource(resource: Resource, aclsSet: Set[Acl], expectedVersion: Int): (Boolean, Int) = { def set(aclData: Array[Byte], expectedVersion: Int): SetDataResponse = { val setDataRequest = SetDataRequest(ResourceZNode.path(resource), aclData, expectedVersion) retryRequestUntilConnected(setDataRequest) } + if (expectedVersion < 0) + throw new IllegalArgumentException(s"Invalid version $expectedVersion provided for conditional update") + + val aclData = ResourceZNode.encode(aclsSet) + + val setDataResponse = set(aclData, expectedVersion) + setDataResponse.resultCode match { + case Code.OK => (true, setDataResponse.stat.getVersion) + case Code.NONODE | Code.BADVERSION => (false, ZkVersion.UnknownVersion) + case _ => throw setDataResponse.resultException.get + } + } + + def createAclsForResourceIfNotExists(resource: Resource, aclsSet: Set[Acl]): (Boolean, Int) = { def create(aclData: Array[Byte]): CreateResponse = { val path = ResourceZNode.path(resource) val createRequest = CreateRequest(path, aclData, acls(path), CreateMode.PERSISTENT) @@ -987,30 +1012,21 @@ class KafkaZkClient private (zooKeeperClient: ZooKeeperClient, isSecure: Boolean val aclData = ResourceZNode.encode(aclsSet) - val setDataResponse = set(aclData, expectedVersion) - setDataResponse.resultCode match { - case Code.OK => (true, setDataResponse.stat.getVersion) - case Code.NONODE => { - val createResponse = create(aclData) - createResponse.resultCode match { - case Code.OK => (true, 0) - case Code.NODEEXISTS => (false, 0) - case _ => throw createResponse.resultException.get - } - } - case Code.BADVERSION => (false, 0) - case _ => throw setDataResponse.resultException.get + val createResponse = create(aclData) + createResponse.resultCode match { + case Code.OK => (true, 0) + case Code.NODEEXISTS => (false, ZkVersion.UnknownVersion) + case _ => throw createResponse.resultException.get } } /** - * Creates Acl change notification message - * @param resource resource name + * Creates an Acl change notification message. + * @param resource resource pattern that has changed */ def createAclChangeNotification(resource: Resource): Unit = { - val store = ZkAclStore(resource.resourceNameType) - val path = store.changeSequenceZNode.createPath - val createRequest = CreateRequest(path, AclChangeNotificationSequenceZNode.encode(resource), acls(path), CreateMode.PERSISTENT_SEQUENTIAL) + val aclChange = ZkAclStore(resource.patternType).changeStore.createChangeNode(resource) + val createRequest = CreateRequest(aclChange.path, aclChange.bytes, acls(aclChange.path), CreateMode.PERSISTENT_SEQUENTIAL) val createResponse = retryRequestUntilConnected(createRequest) createResponse.maybeThrow } @@ -1033,10 +1049,10 @@ class KafkaZkClient private (zooKeeperClient: ZooKeeperClient, isSecure: Boolean * @throws KeeperException if there is an error while deleting Acl change notifications */ def deleteAclChangeNotifications(): Unit = { - ZkAclStore.stores.foreach(store => { + ZkAclChangeStore.stores.foreach(store => { val getChildrenResponse = retryRequestUntilConnected(GetChildrenRequest(store.aclChangePath)) if (getChildrenResponse.resultCode == Code.OK) { - deleteAclChangeNotifications(store, getChildrenResponse.children) + deleteAclChangeNotifications(store.aclChangePath, getChildrenResponse.children) } else if (getChildrenResponse.resultCode != Code.NONODE) { getChildrenResponse.maybeThrow } @@ -1044,13 +1060,14 @@ class KafkaZkClient private (zooKeeperClient: ZooKeeperClient, isSecure: Boolean } /** - * Deletes the Acl change notifications associated with the given sequence nodes - * @param sequenceNodes - */ - private def deleteAclChangeNotifications(store: ZkAclStore, sequenceNodes: Seq[String]): Unit = { - val aclChangeNotificationSequenceZNode = store.changeSequenceZNode + * Deletes the Acl change notifications associated with the given sequence nodes + * + * @param aclChangePath the root path + * @param sequenceNodes the name of the node to delete. + */ + private def deleteAclChangeNotifications(aclChangePath: String, sequenceNodes: Seq[String]): Unit = { val deleteRequests = sequenceNodes.map { sequenceNode => - DeleteRequest(aclChangeNotificationSequenceZNode.deletePath(sequenceNode), ZkVersion.NoVersion) + DeleteRequest(s"$aclChangePath/$sequenceNode", ZkVersion.MatchAnyVersion) } val deleteResponses = retryRequestsUntilConnected(deleteRequests) @@ -1062,22 +1079,22 @@ class KafkaZkClient private (zooKeeperClient: ZooKeeperClient, isSecure: Boolean } /** - * Gets the resource types, for which ACLs are stored, for the supplied resource name type. - * @param nameType The resource name type to retrieve the names for. + * Gets the resource types, for which ACLs are stored, for the supplied resource pattern type. + * @param patternType The resource pattern type to retrieve the names for. * @return list of resource type names */ - def getResourceTypes(nameType: ResourceNameType): Seq[String] = { - getChildren(ZkAclStore(nameType).aclPath) + def getResourceTypes(patternType: PatternType): Seq[String] = { + getChildren(ZkAclStore(patternType).aclPath) } /** - * Gets the resource names, for which ACLs are stored, for a given resource type and name type - * @param nameType The resource name type to retrieve the names for. + * Gets the resource names, for which ACLs are stored, for a given resource type and pattern type + * @param patternType The resource pattern type to retrieve the names for. * @param resourceType Resource type to retrieve the names for. * @return list of resource names */ - def getResourceNames(nameType: ResourceNameType, resourceType: ResourceType): Seq[String] = { - getChildren(ZkAclStore(nameType).path(resourceType)) + def getResourceNames(patternType: PatternType, resourceType: ResourceType): Seq[String] = { + getChildren(ZkAclStore(patternType).path(resourceType)) } /** @@ -1152,7 +1169,7 @@ class KafkaZkClient private (zooKeeperClient: ZooKeeperClient, isSecure: Boolean def setOrCreateDelegationToken(token: DelegationToken): Unit = { def set(tokenData: Array[Byte]): SetDataResponse = { - val setDataRequest = SetDataRequest(DelegationTokenInfoZNode.path(token.tokenInfo().tokenId()), tokenData, ZkVersion.NoVersion) + val setDataRequest = SetDataRequest(DelegationTokenInfoZNode.path(token.tokenInfo().tokenId()), tokenData, ZkVersion.MatchAnyVersion) retryRequestUntilConnected(setDataRequest) } @@ -1334,7 +1351,7 @@ class KafkaZkClient private (zooKeeperClient: ZooKeeperClient, isSecure: Boolean * @return sequence number as the broker id */ def generateBrokerSequenceId(): Int = { - val setDataRequest = SetDataRequest(BrokerSequenceIdZNode.path, Array.empty[Byte], -1) + val setDataRequest = SetDataRequest(BrokerSequenceIdZNode.path, Array.empty[Byte], ZkVersion.MatchAnyVersion) val setDataResponse = retryRequestUntilConnected(setDataRequest) setDataResponse.resultCode match { case Code.OK => setDataResponse.stat.getVersion @@ -1363,7 +1380,7 @@ class KafkaZkClient private (zooKeeperClient: ZooKeeperClient, isSecure: Boolean private def setConsumerOffset(group: String, topicPartition: TopicPartition, offset: Long): SetDataResponse = { val setDataRequest = SetDataRequest(ConsumerOffset.path(group, topicPartition.topic, topicPartition.partition), - ConsumerOffset.encode(offset), ZkVersion.NoVersion) + ConsumerOffset.encode(offset), ZkVersion.MatchAnyVersion) retryRequestUntilConnected(setDataRequest) } @@ -1383,7 +1400,7 @@ class KafkaZkClient private (zooKeeperClient: ZooKeeperClient, isSecure: Boolean getChildrenResponse.resultCode match { case Code.OK => getChildrenResponse.children.foreach(child => deleteRecursive(s"$path/$child")) - val deleteResponse = retryRequestUntilConnected(DeleteRequest(path, ZkVersion.NoVersion)) + val deleteResponse = retryRequestUntilConnected(DeleteRequest(path, ZkVersion.MatchAnyVersion)) if (deleteResponse.resultCode != Code.OK && deleteResponse.resultCode != Code.NONODE) { throw deleteResponse.resultException.get } diff --git a/core/src/main/scala/kafka/zk/ZkData.scala b/core/src/main/scala/kafka/zk/ZkData.scala index 0524b4599c11d..760bd67299dbf 100644 --- a/core/src/main/scala/kafka/zk/ZkData.scala +++ b/core/src/main/scala/kafka/zk/ZkData.scala @@ -23,14 +23,17 @@ import com.fasterxml.jackson.annotation.JsonProperty import com.fasterxml.jackson.core.JsonProcessingException import kafka.api.{ApiVersion, KAFKA_0_10_0_IV1, LeaderAndIsr} import kafka.cluster.{Broker, EndPoint} -import kafka.common.KafkaException +import kafka.common.{NotificationHandler, ZkNodeChangeNotificationListener} import kafka.controller.{IsrChangeNotificationHandler, LeaderIsrAndControllerEpoch} +import kafka.security.auth.Resource.Separator import kafka.security.auth.SimpleAclAuthorizer.VersionedAcls -import kafka.security.auth.{Acl, Literal, Prefixed, Resource, ResourceNameType, ResourceType} +import kafka.security.auth.{Acl, Resource, ResourceType} import kafka.server.{ConfigType, DelegationTokenManager} import kafka.utils.Json -import org.apache.kafka.common.TopicPartition +import org.apache.kafka.common.{KafkaException, TopicPartition} +import org.apache.kafka.common.errors.UnsupportedVersionException import org.apache.kafka.common.network.ListenerName +import org.apache.kafka.common.resource.PatternType import org.apache.kafka.common.security.auth.SecurityProtocol import org.apache.kafka.common.security.token.delegation.{DelegationToken, TokenInformation} import org.apache.kafka.common.utils.Time @@ -41,6 +44,7 @@ import scala.beans.BeanProperty import scala.collection.JavaConverters._ import scala.collection.mutable.ArrayBuffer import scala.collection.{Seq, breakOut} +import scala.util.{Failure, Success, Try} // This file contains objects for encoding/decoding data stored in ZooKeeper nodes (znodes). @@ -425,14 +429,20 @@ object PreferredReplicaElectionZNode { }.map(_.toSet).getOrElse(Set.empty) } +//old consumer path znode +object ConsumerPathZNode { + def path = "/consumers" +} + object ConsumerOffset { - def path(group: String, topic: String, partition: Integer) = s"/consumers/${group}/offsets/${topic}/${partition}" + def path(group: String, topic: String, partition: Integer) = s"${ConsumerPathZNode.path}/${group}/offsets/${topic}/${partition}" def encode(offset: Long): Array[Byte] = offset.toString.getBytes(UTF_8) def decode(bytes: Array[Byte]): Option[Long] = Option(bytes).map(new String(_, UTF_8).toLong) } object ZkVersion { - val NoVersion = -1 + val MatchAnyVersion = -1 // if used in a conditional set, matches any version (the value should match ZooKeeper codebase) + val UnknownVersion = -2 // Version returned from get if node does not exist (internal constant for Kafka codebase, unused value in ZK) } object ZkStat { @@ -445,74 +455,207 @@ object StateChangeHandlers { } /** - * Acls for resources are stored in ZK under a root node that is determined by the [[ResourceNameType]]. - * Under each [[ResourceNameType]] node there will be one child node per resource type (Topic, Cluster, Group, etc). - * Under each resourceType there will be a unique child for each resource path and the data for that child will contain + * Acls for resources are stored in ZK under two root paths: + *
    + *
  • [[org.apache.kafka.common.resource.PatternType#LITERAL Literal]] patterns are stored under '/kafka-acl'. + * The format is JSON. See [[kafka.zk.ResourceZNode]] for details.
  • + *
  • All other patterns are stored under '/kafka-acl-extended/pattern-type'. + * The format is JSON. See [[kafka.zk.ResourceZNode]] for details.
  • + *
+ * + * Under each root node there will be one child node per resource type (Topic, Cluster, Group, etc). + * Under each resourceType there will be a unique child for each resource pattern and the data for that child will contain * list of its acls as a json object. Following gives an example: * *
+  * // Literal patterns:
   * /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-prefixed-acl/Group/group-1 => {"version": 1, "acls": [ { "host":"host1", "permissionType": "Allow","operation": "Read","principal": "User:alice"}]}
+  *
+  * // Prefixed patterns:
+  * /kafka-acl-extended/PREFIXED/Group/group-1 => {"version": 1, "acls": [ { "host":"host1", "permissionType": "Allow","operation": "Read","principal": "User:alice"}]}
   * 
+ * + * Acl change events are also stored under two paths: + *
    + *
  • [[org.apache.kafka.common.resource.PatternType#LITERAL Literal]] patterns are stored under '/kafka-acl-changes'. + * The format is a UTF8 string in the form: <resource-type>:<resource-name>
  • + *
  • All other patterns are stored under '/kafka-acl-extended-changes' + * The format is JSON, as defined by [[kafka.zk.ExtendedAclChangeEvent]]
  • + *
*/ -case class ZkAclStore(nameType: ResourceNameType) { - val aclPath: String = nameType match { - case Literal => "/kafka-acl" - case Prefixed => "/kafka-prefixed-acl" - case _ => throw new IllegalArgumentException("Unknown name type:" + nameType) +sealed trait ZkAclStore { + val patternType: PatternType + val aclPath: String + + def path(resourceType: ResourceType): String = s"$aclPath/$resourceType" + + def path(resourceType: ResourceType, resourceName: String): String = s"$aclPath/$resourceType/$resourceName" + + def changeStore: ZkAclChangeStore +} + +object ZkAclStore { + private val storesByType: Map[PatternType, ZkAclStore] = PatternType.values + .filter(patternType => patternType != PatternType.MATCH) + .filter(patternType => patternType != PatternType.ANY) + .filter(patternType => patternType != PatternType.UNKNOWN) + .map(patternType => (patternType, create(patternType))) + .toMap + + val stores: Iterable[ZkAclStore] = storesByType.values + + val securePaths: Iterable[String] = stores + .flatMap(store => Set(store.aclPath, store.changeStore.aclChangePath)) + + def apply(patternType: PatternType): ZkAclStore = { + storesByType.get(patternType) match { + case Some(store) => store + case None => throw new KafkaException(s"Invalid pattern type: $patternType") + } } - val aclChangePath: String = nameType match { - case Literal => "/kafka-acl-changes" - case Prefixed => "/kafka-prefixed-acl-changes" - case _ => throw new IllegalArgumentException("Unknown name type:" + nameType) + private def create(patternType: PatternType) = { + patternType match { + case PatternType.LITERAL => LiteralAclStore + case _ => new ExtendedAclStore(patternType) + } } +} - def path(resourceType: ResourceType) = s"$aclPath/$resourceType" +object LiteralAclStore extends ZkAclStore { + val patternType: PatternType = PatternType.LITERAL + val aclPath: String = "/kafka-acl" - def path(resourceType: ResourceType, resourceName: String): String = s"$aclPath/$resourceType/$resourceName" + def changeStore: ZkAclChangeStore = LiteralAclChangeStore +} - def changeSequenceZNode: AclChangeNotificationSequenceZNode = AclChangeNotificationSequenceZNode(this) +class ExtendedAclStore(val patternType: PatternType) extends ZkAclStore { + if (patternType == PatternType.LITERAL) + throw new IllegalArgumentException("Literal pattern types are not supported") - def decode(notificationMessage: Array[Byte]): Resource = AclChangeNotificationSequenceZNode.decode(nameType, notificationMessage) + val aclPath: String = s"/kafka-acl-extended/${patternType.name.toLowerCase}" + + def changeStore: ZkAclChangeStore = ExtendedAclChangeStore } -object ZkAclStore { - val stores: Seq[ZkAclStore] = ResourceNameType.values - .map(nameType => ZkAclStore(nameType)) +trait AclChangeNotificationHandler { + def processNotification(resource: Resource): Unit +} - val securePaths: Seq[String] = stores - .flatMap(store => List(store.aclPath, store.aclChangePath)) +trait AclChangeSubscription extends AutoCloseable { + def close(): Unit } -object ResourceZNode { - def path(resource: Resource): String = ZkAclStore(resource.resourceNameType).path(resource.resourceType, resource.name) +case class AclChangeNode(path: String, bytes: Array[Byte]) - def encode(acls: Set[Acl]): Array[Byte] = Json.encodeAsBytes(Acl.toJsonCompatibleMap(acls).asJava) - def decode(bytes: Array[Byte], stat: Stat): VersionedAcls = VersionedAcls(Acl.fromBytes(bytes), stat.getVersion) +sealed trait ZkAclChangeStore { + val aclChangePath: String + def createPath: String = s"$aclChangePath/${ZkAclChangeStore.SequenceNumberPrefix}" + + def decode(bytes: Array[Byte]): Resource + + protected def encode(resource: Resource): Array[Byte] + + def createChangeNode(resource: Resource): AclChangeNode = AclChangeNode(createPath, encode(resource)) + + def createListener(handler: AclChangeNotificationHandler, zkClient: KafkaZkClient): AclChangeSubscription = { + val rawHandler: NotificationHandler = new NotificationHandler { + def processNotification(bytes: Array[Byte]): Unit = + handler.processNotification(decode(bytes)) + } + + val aclChangeListener = new ZkNodeChangeNotificationListener( + zkClient, aclChangePath, ZkAclChangeStore.SequenceNumberPrefix, rawHandler) + + aclChangeListener.init() + + new AclChangeSubscription { + def close(): Unit = aclChangeListener.close() + } + } } -object AclChangeNotificationSequenceZNode { - val Separator = ":" +object ZkAclChangeStore { + val stores: Iterable[ZkAclChangeStore] = List(LiteralAclChangeStore, ExtendedAclChangeStore) + def SequenceNumberPrefix = "acl_changes_" +} + +case object LiteralAclChangeStore extends ZkAclChangeStore { + val name = "LiteralAclChangeStore" + val aclChangePath: String = "/kafka-acl-changes" + + def encode(resource: Resource): Array[Byte] = { + if (resource.patternType != PatternType.LITERAL) + throw new IllegalArgumentException("Only literal resource patterns can be encoded") + + val legacyName = resource.resourceType + Resource.Separator + resource.name + legacyName.getBytes(UTF_8) + } + + def decode(bytes: Array[Byte]): Resource = { + val string = new String(bytes, UTF_8) + string.split(Separator, 2) match { + case Array(resourceType, resourceName, _*) => new Resource(ResourceType.fromString(resourceType), resourceName, PatternType.LITERAL) + case _ => throw new IllegalArgumentException("expected a string in format ResourceType:ResourceName but got " + string) + } + } +} + +case object ExtendedAclChangeStore extends ZkAclChangeStore { + val name = "ExtendedAclChangeStore" + val aclChangePath: String = "/kafka-acl-extended-changes" def encode(resource: Resource): Array[Byte] = { - (resource.resourceType.name + Separator + resource.name).getBytes(UTF_8) + if (resource.patternType == PatternType.LITERAL) + throw new IllegalArgumentException("Literal pattern types are not supported") + + Json.encodeAsBytes(ExtendedAclChangeEvent( + ExtendedAclChangeEvent.currentVersion, + resource.resourceType.name, + resource.name, + resource.patternType.name)) } - def decode(nameType: ResourceNameType, bytes: Array[Byte]): Resource = { - val str = new String(bytes, UTF_8) - str.split(Separator, 2) match { - case Array(resourceType, name, _*) => Resource(ResourceType.fromString(resourceType), name, nameType) - case _ => throw new IllegalArgumentException("expected a string in format ResourceType:ResourceName but got " + str) + def decode(bytes: Array[Byte]): Resource = { + val changeEvent = Json.parseBytesAs[ExtendedAclChangeEvent](bytes) match { + case Right(event) => event + case Left(e) => throw new IllegalArgumentException("Failed to parse ACL change event", e) + } + + changeEvent.toResource match { + case Success(r) => r + case Failure(e) => throw new IllegalArgumentException("Failed to convert ACL change event to resource", e) } } } -case class AclChangeNotificationSequenceZNode(store: ZkAclStore) { - def createPath = s"${store.aclChangePath}/${AclChangeNotificationSequenceZNode.SequenceNumberPrefix}" - def deletePath(sequenceNode: String) = s"${store.aclChangePath}/$sequenceNode" +object ResourceZNode { + def path(resource: Resource): String = ZkAclStore(resource.patternType).path(resource.resourceType, resource.name) + + def encode(acls: Set[Acl]): Array[Byte] = Json.encodeAsBytes(Acl.toJsonCompatibleMap(acls).asJava) + def decode(bytes: Array[Byte], stat: Stat): VersionedAcls = VersionedAcls(Acl.fromBytes(bytes), stat.getVersion) +} + +object ExtendedAclChangeEvent { + val currentVersion: Int = 1 +} + +case class ExtendedAclChangeEvent(@BeanProperty @JsonProperty("version") version: Int, + @BeanProperty @JsonProperty("resourceType") resourceType: String, + @BeanProperty @JsonProperty("name") name: String, + @BeanProperty @JsonProperty("patternType") patternType: String) { + if (version > ExtendedAclChangeEvent.currentVersion) + throw new UnsupportedVersionException(s"Acl change event received for unsupported version: $version") + + def toResource: Try[Resource] = { + for { + resType <- Try(ResourceType.fromString(resourceType)) + patType <- Try(PatternType.fromString(patternType)) + resource = Resource(resType, name, patType) + } yield resource + } } object ClusterZNode { @@ -583,7 +726,7 @@ object ZkData { // These are persistent ZK paths that should exist on kafka broker startup. val PersistentZkPaths = Seq( - "/consumers", // old consumer path + ConsumerPathZNode.path, // old consumer path BrokerIdsZNode.path, TopicsZNode.path, ConfigEntityChangeNotificationZNode.path, @@ -605,7 +748,8 @@ object ZkData { } def defaultAcls(isSecure: Boolean, path: String): Seq[ACL] = { - if (isSecure) { + //Old Consumer path is kept open as different consumers will write under this node. + if (!ConsumerPathZNode.path.equals(path) && isSecure) { val acls = new ArrayBuffer[ACL] acls ++= ZooDefs.Ids.CREATOR_ALL_ACL.asScala if (!sensitivePath(path)) diff --git a/core/src/main/scala/kafka/zookeeper/ZooKeeperClient.scala b/core/src/main/scala/kafka/zookeeper/ZooKeeperClient.scala index 5cb127c3c7edf..97ec9a44c36f3 100755 --- a/core/src/main/scala/kafka/zookeeper/ZooKeeperClient.scala +++ b/core/src/main/scala/kafka/zookeeper/ZooKeeperClient.scala @@ -92,7 +92,12 @@ class ZooKeeperClient(connectString: String, metricNames += "SessionState" expiryScheduler.startup() - waitUntilConnected(connectionTimeoutMs, TimeUnit.MILLISECONDS) + try waitUntilConnected(connectionTimeoutMs, TimeUnit.MILLISECONDS) + catch { + case e: Throwable => + close() + throw e + } override def metricName(name: String, metricTags: scala.collection.Map[String, String]): MetricName = { explicitMetricName(metricGroup, metricType, name, metricTags) diff --git a/core/src/test/scala/integration/kafka/api/AdminClientIntegrationTest.scala b/core/src/test/scala/integration/kafka/api/AdminClientIntegrationTest.scala index 331a4491e0d63..75f481d2b3c2b 100644 --- a/core/src/test/scala/integration/kafka/api/AdminClientIntegrationTest.scala +++ b/core/src/test/scala/integration/kafka/api/AdminClientIntegrationTest.scala @@ -29,18 +29,19 @@ import kafka.log.LogConfig import kafka.server.{Defaults, KafkaConfig, KafkaServer} import org.apache.kafka.clients.admin._ import kafka.utils.{Logging, TestUtils} +import kafka.utils.TestUtils._ import kafka.utils.Implicits._ import org.apache.kafka.clients.admin.NewTopic import org.apache.kafka.clients.consumer.{ConsumerConfig, KafkaConsumer} import org.apache.kafka.clients.producer.KafkaProducer import org.apache.kafka.clients.producer.ProducerRecord -import org.apache.kafka.common.{ConsumerGroupState, KafkaFuture, TopicPartition, TopicPartitionReplica} +import org.apache.kafka.common.{ConsumerGroupState, TopicPartition, TopicPartitionReplica} import org.apache.kafka.common.acl._ import org.apache.kafka.common.config.ConfigResource import org.apache.kafka.common.errors._ import org.junit.{After, Before, Rule, Test} import org.apache.kafka.common.requests.{DeleteRecordsRequest, MetadataResponse} -import org.apache.kafka.common.resource.{Resource, ResourceNameType, ResourceType} +import org.apache.kafka.common.resource.{PatternType, ResourcePattern, ResourceType} import org.junit.rules.Timeout import org.junit.Assert._ @@ -125,18 +126,6 @@ class AdminClientIntegrationTest extends IntegrationTestHarness with Logging { }, "timed out waiting for topics") } - def assertFutureExceptionTypeEquals(future: KafkaFuture[_], clazz: Class[_ <: Throwable]): Unit = { - try { - future.get() - fail("Expected CompletableFuture.get to return an exception") - } catch { - case e: ExecutionException => - val cause = e.getCause() - assertTrue("Expected an exception of type " + clazz.getName + "; got type " + - cause.getClass().getName, clazz.isInstance(cause)) - } - } - @Test def testClose(): Unit = { val client = AdminClient.create(createConfig()) @@ -353,7 +342,7 @@ class AdminClientIntegrationTest extends IntegrationTestHarness with Logging { val numMessages = new AtomicInteger import scala.concurrent.ExecutionContext.Implicits._ val producerFuture = Future { - val producer = TestUtils.createNewProducer( + val producer = TestUtils.createProducer( TestUtils.getBrokerListStrFromServers(servers, protocol = securityProtocol), securityProtocol = securityProtocol, trustStoreFile = trustStoreFile, @@ -389,7 +378,8 @@ class AdminClientIntegrationTest extends IntegrationTestHarness with Logging { val finalNumMessages = Await.result(producerFuture, Duration(20, TimeUnit.SECONDS)) // Verify that all messages that are produced can be consumed - val consumerRecords = TestUtils.consumeTopicRecords(servers, topic, finalNumMessages, securityProtocol, trustStoreFile) + val consumerRecords = TestUtils.consumeTopicRecords(servers, topic, finalNumMessages, + securityProtocol = securityProtocol, trustStoreFile = trustStoreFile) consumerRecords.zipWithIndex.foreach { case (consumerRecord, index) => assertEquals(s"xxxxxxxxxxxxxxxxxxxx-$index", new String(consumerRecord.value)) } @@ -811,6 +801,83 @@ class AdminClientIntegrationTest extends IntegrationTestHarness with Logging { assertEquals(3, servers(i).replicaManager.getReplica(topicPartition).get.logStartOffset) } + @Test + def testReplicaCanFetchFromLogStartOffsetAfterDeleteRecords(): Unit = { + val leaders = createTopic(topic, numPartitions = 1, replicationFactor = serverCount) + val followerIndex = if (leaders(0) != servers(0).config.brokerId) 0 else 1 + + def waitForFollowerLog(expectedStartOffset: Long, expectedEndOffset: Long): Unit = { + TestUtils.waitUntilTrue(() => servers(followerIndex).replicaManager.getReplica(topicPartition) != None, + "Expected follower to create replica for partition") + + // wait until the follower discovers that log start offset moved beyond its HW + TestUtils.waitUntilTrue(() => { + servers(followerIndex).replicaManager.getReplica(topicPartition).get.logStartOffset == expectedStartOffset + }, s"Expected follower to discover new log start offset $expectedStartOffset") + + TestUtils.waitUntilTrue(() => { + servers(followerIndex).replicaManager.getReplica(topicPartition).get.logEndOffset.messageOffset == expectedEndOffset + }, s"Expected follower to catch up to log end offset $expectedEndOffset") + } + + // we will produce to topic and delete records while one follower is down + killBroker(followerIndex) + + client = AdminClient.create(createConfig) + sendRecords(producers.head, 100, topicPartition) + + val result = client.deleteRecords(Map(topicPartition -> RecordsToDelete.beforeOffset(3L)).asJava) + result.all().get() + + // start the stopped broker to verify that it will be able to fetch from new log start offset + restartDeadBrokers() + + waitForFollowerLog(expectedStartOffset=3L, expectedEndOffset=100L) + + // after the new replica caught up, all replicas should have same log start offset + for (i <- 0 until serverCount) + assertEquals(3, servers(i).replicaManager.getReplica(topicPartition).get.logStartOffset) + + // kill the same follower again, produce more records, and delete records beyond follower's LOE + killBroker(followerIndex) + sendRecords(producers.head, 100, topicPartition) + val result1 = client.deleteRecords(Map(topicPartition -> RecordsToDelete.beforeOffset(117L)).asJava) + result1.all().get() + restartDeadBrokers() + waitForFollowerLog(expectedStartOffset=117L, expectedEndOffset=200L) + } + + @Test + def testAlterLogDirsAfterDeleteRecords(): Unit = { + client = AdminClient.create(createConfig) + createTopic(topic, numPartitions = 1, replicationFactor = serverCount) + val expectedLEO = 100 + sendRecords(producers.head, expectedLEO, topicPartition) + + // delete records to move log start offset + val result = client.deleteRecords(Map(topicPartition -> RecordsToDelete.beforeOffset(3L)).asJava) + result.all().get() + // make sure we are in the expected state after delete records + for (i <- 0 until serverCount) { + assertEquals(3, servers(i).replicaManager.getReplica(topicPartition).get.logStartOffset) + assertEquals(expectedLEO, servers(i).replicaManager.getReplica(topicPartition).get.logEndOffset.messageOffset) + } + + // we will create another dir just for one server + val futureLogDir = servers(0).config.logDirs(1) + val futureReplica = new TopicPartitionReplica(topic, 0, servers(0).config.brokerId) + + // Verify that replica can be moved to the specified log directory + client.alterReplicaLogDirs(Map(futureReplica -> futureLogDir).asJava).all.get + TestUtils.waitUntilTrue(() => { + futureLogDir == servers(0).logManager.getLog(topicPartition).get.dir.getParent + }, "timed out waiting for replica movement") + + // once replica moved, its LSO and LEO should match other replicas + assertEquals(3, servers(0).replicaManager.getReplica(topicPartition).get.logStartOffset) + assertEquals(expectedLEO, servers(0).replicaManager.getReplica(topicPartition).get.logEndOffset.messageOffset) + } + @Test def testOffsetsForTimesAfterDeleteRecords(): Unit = { createTopic(topic, numPartitions = 2, replicationFactor = serverCount) @@ -933,7 +1000,7 @@ class AdminClientIntegrationTest extends IntegrationTestHarness with Logging { checkInvalidAlterConfigs(zkClient, servers, client) } - val ACL1 = new AclBinding(new Resource(ResourceType.TOPIC, "mytopic3", ResourceNameType.LITERAL), + val ACL1 = new AclBinding(new ResourcePattern(ResourceType.TOPIC, "mytopic3", PatternType.LITERAL), new AccessControlEntry("User:ANONYMOUS", "*", AclOperation.DESCRIBE, AclPermissionType.ALLOW)) /** @@ -975,7 +1042,7 @@ class AdminClientIntegrationTest extends IntegrationTestHarness with Logging { @Test def testForceClose(): Unit = { val config = createConfig() - config.put(AdminClientConfig.BOOTSTRAP_SERVERS_CONFIG, "localhost:22") + config.put(AdminClientConfig.BOOTSTRAP_SERVERS_CONFIG, s"localhost:${TestUtils.IncorrectBrokerPort}") client = AdminClient.create(config) // Because the bootstrap servers are set up incorrectly, this call will not complete, but must be // cancelled by the close operation. @@ -992,7 +1059,7 @@ class AdminClientIntegrationTest extends IntegrationTestHarness with Logging { @Test def testMinimumRequestTimeouts(): Unit = { val config = createConfig() - config.put(AdminClientConfig.BOOTSTRAP_SERVERS_CONFIG, "localhost:22") + config.put(AdminClientConfig.BOOTSTRAP_SERVERS_CONFIG, s"localhost:${TestUtils.IncorrectBrokerPort}") config.put(AdminClientConfig.REQUEST_TIMEOUT_MS_CONFIG, "0") client = AdminClient.create(config) val startTimeMs = Time.SYSTEM.milliseconds() @@ -1000,7 +1067,7 @@ class AdminClientIntegrationTest extends IntegrationTestHarness with Logging { new CreateTopicsOptions().timeoutMs(2)).all() assertFutureExceptionTypeEquals(future, classOf[TimeoutException]) val endTimeMs = Time.SYSTEM.milliseconds() - assertTrue("Expected the timeout to take at least one millisecond.", endTimeMs > startTimeMs); + assertTrue("Expected the timeout to take at least one millisecond.", endTimeMs > startTimeMs) } /** @@ -1011,7 +1078,7 @@ class AdminClientIntegrationTest extends IntegrationTestHarness with Logging { val config = createConfig() config.put(AdminClientConfig.REQUEST_TIMEOUT_MS_CONFIG, "100000000") val factory = new KafkaAdminClientTest.FailureInjectingTimeoutProcessorFactory() - val client = KafkaAdminClientTest.createInternal(new AdminClientConfig(config), factory) + client = KafkaAdminClientTest.createInternal(new AdminClientConfig(config), factory) val future = client.createTopics(Seq("mytopic", "mytopic2").map(new NewTopic(_, 1, 1)).asJava, new CreateTopicsOptions().validateOnly(true)).all() assertFutureExceptionTypeEquals(future, classOf[TimeoutException]) @@ -1027,7 +1094,7 @@ class AdminClientIntegrationTest extends IntegrationTestHarness with Logging { @Test def testConsumerGroups(): Unit = { val config = createConfig() - val client = AdminClient.create(config) + client = AdminClient.create(config) try { // Verify that initially there are no consumer groups to list. val list1 = client.listConsumerGroups() @@ -1040,7 +1107,7 @@ class AdminClientIntegrationTest extends IntegrationTestHarness with Logging { new NewTopic(testTopicName, testNumPartitions, 1))).all().get() waitForTopics(client, List(testTopicName), List()) - val producer = createNewProducer + val producer = createProducer try { producer.send(new ProducerRecord(testTopicName, 0, null, null)).get() } finally { @@ -1052,7 +1119,7 @@ class AdminClientIntegrationTest extends IntegrationTestHarness with Logging { val newConsumerConfig = new Properties(consumerConfig) newConsumerConfig.setProperty(ConsumerConfig.GROUP_ID_CONFIG, testGroupId) newConsumerConfig.setProperty(ConsumerConfig.CLIENT_ID_CONFIG, testClientId) - val consumer = TestUtils.createNewConsumer(brokerList, + val consumer = TestUtils.createConsumer(brokerList, securityProtocol = this.securityProtocol, trustStoreFile = this.trustStoreFile, saslProperties = this.clientSaslProperties, diff --git a/core/src/test/scala/integration/kafka/api/AuthorizerIntegrationTest.scala b/core/src/test/scala/integration/kafka/api/AuthorizerIntegrationTest.scala index b40dab7809c45..3a53eeda6713f 100644 --- a/core/src/test/scala/integration/kafka/api/AuthorizerIntegrationTest.scala +++ b/core/src/test/scala/integration/kafka/api/AuthorizerIntegrationTest.scala @@ -19,19 +19,19 @@ import java.util.regex.Pattern import java.util.{ArrayList, Collections, Properties} import java.time.Duration -import kafka.admin.AdminClient -import kafka.admin.ConsumerGroupCommand.{ConsumerGroupCommandOptions, KafkaConsumerGroupService} +import kafka.admin.ConsumerGroupCommand.{ConsumerGroupCommandOptions, ConsumerGroupService} import kafka.common.TopicAndPartition import kafka.log.LogConfig import kafka.network.SocketServer import kafka.security.auth._ import kafka.server.{BaseRequestTest, KafkaConfig} import kafka.utils.TestUtils -import org.apache.kafka.clients.admin.NewPartitions +import org.apache.kafka.clients.admin.{AdminClient, AdminClientConfig, NewPartitions} import org.apache.kafka.clients.consumer._ import org.apache.kafka.clients.consumer.internals.NoOpConsumerRebalanceListener import org.apache.kafka.clients.producer._ import org.apache.kafka.common.acl.{AccessControlEntry, AccessControlEntryFilter, AclBinding, AclBindingFilter, AclOperation, AclPermissionType} +import org.apache.kafka.common.config.ConfigResource import org.apache.kafka.common.errors._ import org.apache.kafka.common.internals.Topic.GROUP_METADATA_TOPIC_NAME import org.apache.kafka.common.network.ListenerName @@ -39,10 +39,12 @@ import org.apache.kafka.common.protocol.{ApiKeys, Errors} import org.apache.kafka.common.record.{CompressionType, MemoryRecords, Records, SimpleRecord} import org.apache.kafka.common.requests.CreateAclsRequest.AclCreation import org.apache.kafka.common.requests.CreateTopicsRequest.TopicDetails -import org.apache.kafka.common.requests.{Resource => RResource, ResourceType => RResourceType, _} -import org.apache.kafka.common.resource.{ResourceFilter, ResourceNameType, Resource => AdminResource, ResourceType => AdminResourceType} +import org.apache.kafka.common.requests._ +import org.apache.kafka.common.resource.PatternType.LITERAL +import org.apache.kafka.common.resource.{ResourcePattern, ResourcePatternFilter, ResourceType => AdminResourceType} import org.apache.kafka.common.security.auth.{KafkaPrincipal, SecurityProtocol} import org.apache.kafka.common.{KafkaException, Node, TopicPartition, requests} +import org.apache.kafka.test.{TestUtils => JTestUtils} import org.junit.Assert._ import org.junit.{After, Assert, Before, Test} @@ -70,11 +72,11 @@ class AuthorizerIntegrationTest extends BaseRequestTest { val deleteRecordsPartition = new TopicPartition(deleteTopic, part) val topicAndPartition = TopicAndPartition(topic, part) val group = "my-group" - val topicResource = new Resource(Topic, topic, Literal) - val groupResource = new Resource(Group, group, Literal) - val deleteTopicResource = new Resource(Topic, deleteTopic, Literal) - val transactionalIdResource = new Resource(TransactionalId, transactionalId, Literal) - val createTopicResource = new Resource(Topic, createTopic, Literal) + val topicResource = Resource(Topic, topic, LITERAL) + val groupResource = Resource(Group, group, LITERAL) + val deleteTopicResource = Resource(Topic, deleteTopic, LITERAL) + val transactionalIdResource = Resource(TransactionalId, transactionalId, LITERAL) + val createTopicResource = Resource(Topic, createTopic, LITERAL) val groupReadAcl = Map(groupResource -> Set(new Acl(userPrincipal, Allow, Acl.WildCardHost, Read))) val groupDescribeAcl = Map(groupResource -> Set(new Acl(userPrincipal, Allow, Acl.WildCardHost, Describe))) @@ -104,6 +106,8 @@ class AuthorizerIntegrationTest extends BaseRequestTest { val producerConfig = new Properties val numRecords = 1 + val adminClients = Buffer[AdminClient]() + override def propertyOverrides(properties: Properties): Unit = { properties.put(KafkaConfig.AuthorizerClassNameProp, classOf[SimpleAclAuthorizer].getName) properties.put(KafkaConfig.BrokerIdProp, brokerId.toString) @@ -174,9 +178,9 @@ class AuthorizerIntegrationTest extends BaseRequestTest { ApiKeys.DELETE_RECORDS -> ((resp: requests.DeleteRecordsResponse) => resp.responses.get(deleteRecordsPartition).error), ApiKeys.OFFSET_FOR_LEADER_EPOCH -> ((resp: OffsetsForLeaderEpochResponse) => resp.responses.get(tp).error), ApiKeys.DESCRIBE_CONFIGS -> ((resp: DescribeConfigsResponse) => - resp.configs.get(new RResource(RResourceType.TOPIC, tp.topic)).error.error), + resp.configs.get(new ConfigResource(ConfigResource.Type.TOPIC, tp.topic)).error.error), ApiKeys.ALTER_CONFIGS -> ((resp: AlterConfigsResponse) => - resp.errors.get(new RResource(RResourceType.TOPIC, tp.topic)).error), + resp.errors.get(new ConfigResource(ConfigResource.Type.TOPIC, tp.topic)).error), ApiKeys.INIT_PRODUCER_ID -> ((resp: InitProducerIdResponse) => resp.error), ApiKeys.WRITE_TXN_MARKERS -> ((resp: WriteTxnMarkersResponse) => resp.errors(producerId).get(tp)), ApiKeys.ADD_PARTITIONS_TO_TXN -> ((resp: AddPartitionsToTxnResponse) => resp.errors.get(tp)), @@ -238,12 +242,12 @@ class AuthorizerIntegrationTest extends BaseRequestTest { addAndVerifyAcls(Set(new Acl(userPrincipal, Allow, Acl.WildCardHost, ClusterAction)), Resource.ClusterResource) for (_ <- 0 until producerCount) - producers += TestUtils.createNewProducer(TestUtils.getBrokerListStrFromServers(servers), + producers += TestUtils.createProducer(TestUtils.getBrokerListStrFromServers(servers), maxBlockMs = 3000, acks = 1) for (_ <- 0 until consumerCount) - consumers += TestUtils.createNewConsumer(TestUtils.getBrokerListStrFromServers(servers), groupId = group, securityProtocol = SecurityProtocol.PLAINTEXT) + consumers += TestUtils.createConsumer(TestUtils.getBrokerListStrFromServers(servers), groupId = group, securityProtocol = SecurityProtocol.PLAINTEXT) // create the consumer offset topic createTopic(GROUP_METADATA_TOPIC_NAME, topicConfig = servers.head.groupCoordinator.offsetsTopicConfigs) @@ -257,6 +261,7 @@ class AuthorizerIntegrationTest extends BaseRequestTest { producers.foreach(_.close()) consumers.foreach(_.wakeup()) consumers.foreach(_.close()) + adminClients.foreach(_.close()) removeAllAcls() super.tearDown() } @@ -365,11 +370,11 @@ class AuthorizerIntegrationTest extends BaseRequestTest { private def deleteRecordsRequest = new DeleteRecordsRequest.Builder(5000, Collections.singletonMap(deleteRecordsPartition, 0L)).build() private def describeConfigsRequest = - new DescribeConfigsRequest.Builder(Collections.singleton(new RResource(RResourceType.TOPIC, tp.topic))).build() + new DescribeConfigsRequest.Builder(Collections.singleton(new ConfigResource(ConfigResource.Type.TOPIC, tp.topic))).build() private def alterConfigsRequest = new AlterConfigsRequest.Builder( - Collections.singletonMap(new RResource(RResourceType.TOPIC, tp.topic), + Collections.singletonMap(new ConfigResource(ConfigResource.Type.TOPIC, tp.topic), new AlterConfigsRequest.Config(Collections.singleton( new AlterConfigsRequest.ConfigEntry(LogConfig.MaxMessageBytesProp, "1000000") ))), true).build() @@ -378,12 +383,12 @@ class AuthorizerIntegrationTest extends BaseRequestTest { private def createAclsRequest = new CreateAclsRequest.Builder( Collections.singletonList(new AclCreation(new AclBinding( - new AdminResource(AdminResourceType.TOPIC, "mytopic"), + new ResourcePattern(AdminResourceType.TOPIC, "mytopic", LITERAL), new AccessControlEntry(userPrincipal.toString, "*", AclOperation.WRITE, AclPermissionType.DENY))))).build() private def deleteAclsRequest = new DeleteAclsRequest.Builder( Collections.singletonList(new AclBindingFilter( - new ResourceFilter(AdminResourceType.TOPIC, null, ResourceNameType.LITERAL), + new ResourcePatternFilter(AdminResourceType.TOPIC, null, LITERAL), new AccessControlEntryFilter(userPrincipal.toString, "*", AclOperation.ANY, AclPermissionType.DENY)))).build() private def alterReplicaLogDirsRequest = new AlterReplicaLogDirsRequest.Builder(Collections.singletonMap(tp, logDir)).build() @@ -577,13 +582,13 @@ class AuthorizerIntegrationTest extends BaseRequestTest { private def testCreatePermissionNeededToWriteToNonExistentTopic(resType: ResourceType) { val topicPartition = new TopicPartition(createTopic, 0) - val newTopicResource = new Resource(Topic, createTopic, Literal) + val newTopicResource = Resource(Topic, createTopic, LITERAL) addAndVerifyAcls(Set(new Acl(userPrincipal, Allow, Acl.WildCardHost, Write)), newTopicResource) try { sendRecords(numRecords, topicPartition) Assert.fail("should have thrown exception") } catch { - case e: TopicAuthorizationException => + case e: TopicAuthorizationException => assertEquals(Collections.singleton(createTopic), e.unauthorizedTopics()) } @@ -733,7 +738,7 @@ class AuthorizerIntegrationTest extends BaseRequestTest { // create an unmatched topic val unmatchedTopic = "unmatched" createTopic(unmatchedTopic) - addAndVerifyAcls(Set(new Acl(userPrincipal, Allow, Acl.WildCardHost, Write)), new Resource(Topic, unmatchedTopic, Literal)) + addAndVerifyAcls(Set(new Acl(userPrincipal, Allow, Acl.WildCardHost, Write)), Resource(Topic, unmatchedTopic, LITERAL)) sendRecords(1, new TopicPartition(unmatchedTopic, part)) removeAllAcls() @@ -745,8 +750,8 @@ class AuthorizerIntegrationTest extends BaseRequestTest { // set the subscription pattern to an internal topic that the consumer has read permission to. Since // internal topics are not included, we should not be assigned any partitions from this topic - addAndVerifyAcls(Set(new Acl(userPrincipal, Allow, Acl.WildCardHost, Read)), new Resource(Topic, - GROUP_METADATA_TOPIC_NAME, Literal)) + addAndVerifyAcls(Set(new Acl(userPrincipal, Allow, Acl.WildCardHost, Read)), Resource(Topic, + GROUP_METADATA_TOPIC_NAME, LITERAL)) consumer.subscribe(Pattern.compile(GROUP_METADATA_TOPIC_NAME)) consumer.poll(0) assertTrue(consumer.subscription().isEmpty) @@ -764,7 +769,7 @@ class AuthorizerIntegrationTest extends BaseRequestTest { val consumerConfig = new Properties consumerConfig.put(ConsumerConfig.EXCLUDE_INTERNAL_TOPICS_CONFIG, "false") - val consumer = TestUtils.createNewConsumer(TestUtils.getBrokerListStrFromServers(servers), groupId = group, + val consumer = TestUtils.createConsumer(TestUtils.getBrokerListStrFromServers(servers), groupId = group, securityProtocol = SecurityProtocol.PLAINTEXT, props = Some(consumerConfig)) try { // ensure that internal topics are not included if no permission @@ -774,7 +779,7 @@ class AuthorizerIntegrationTest extends BaseRequestTest { // now authorize the user for the internal topic and verify that we can subscribe addAndVerifyAcls(Set(new Acl(userPrincipal, Allow, Acl.WildCardHost, Read)), Resource(Topic, - GROUP_METADATA_TOPIC_NAME, Literal)) + GROUP_METADATA_TOPIC_NAME, LITERAL)) consumer.subscribe(Pattern.compile(GROUP_METADATA_TOPIC_NAME)) consumer.poll(0) assertEquals(Set(GROUP_METADATA_TOPIC_NAME), consumer.subscription.asScala) @@ -789,12 +794,12 @@ class AuthorizerIntegrationTest extends BaseRequestTest { addAndVerifyAcls(Set(new Acl(userPrincipal, Allow, Acl.WildCardHost, Read)), topicResource) addAndVerifyAcls(Set(new Acl(userPrincipal, Allow, Acl.WildCardHost, Read)), groupResource) - val internalTopicResource = new Resource(Topic, GROUP_METADATA_TOPIC_NAME, Literal) + val internalTopicResource = Resource(Topic, GROUP_METADATA_TOPIC_NAME, LITERAL) addAndVerifyAcls(Set(new Acl(userPrincipal, Allow, Acl.WildCardHost, Describe)), internalTopicResource) val consumerConfig = new Properties consumerConfig.put(ConsumerConfig.EXCLUDE_INTERNAL_TOPICS_CONFIG, "false") - val consumer = TestUtils.createNewConsumer(TestUtils.getBrokerListStrFromServers(servers), groupId = group, + val consumer = TestUtils.createConsumer(TestUtils.getBrokerListStrFromServers(servers), groupId = group, securityProtocol = SecurityProtocol.PLAINTEXT, props = Some(consumerConfig)) try { consumer.subscribe(Pattern.compile(".*")) @@ -818,7 +823,7 @@ class AuthorizerIntegrationTest extends BaseRequestTest { val consumerConfig = new Properties consumerConfig.put(ConsumerConfig.EXCLUDE_INTERNAL_TOPICS_CONFIG, "false") - val consumer = TestUtils.createNewConsumer(TestUtils.getBrokerListStrFromServers(servers), groupId = group, + val consumer = TestUtils.createConsumer(TestUtils.getBrokerListStrFromServers(servers), groupId = group, securityProtocol = SecurityProtocol.PLAINTEXT, props = Some(consumerConfig)) try { consumer.subscribe(Pattern.compile(topicPattern)) @@ -836,13 +841,13 @@ class AuthorizerIntegrationTest extends BaseRequestTest { @Test def testCreatePermissionOnClusterToReadFromNonExistentTopic() { testCreatePermissionNeededToReadFromNonExistentTopic("newTopic", - Set(new Acl(userPrincipal, Allow, Acl.WildCardHost, Create)), + Set(new Acl(userPrincipal, Allow, Acl.WildCardHost, Create)), Cluster) } private def testCreatePermissionNeededToReadFromNonExistentTopic(newTopic: String, acls: Set[Acl], resType: ResourceType) { val topicPartition = new TopicPartition(newTopic, 0) - val newTopicResource = new Resource(Topic, newTopic, Literal) + val newTopicResource = Resource(Topic, newTopic, LITERAL) addAndVerifyAcls(Set(new Acl(userPrincipal, Allow, Acl.WildCardHost, Read)), newTopicResource) addAndVerifyAcls(groupReadAcl(groupResource), groupResource) this.consumers.head.assign(List(topicPartition).asJava) @@ -860,6 +865,31 @@ class AuthorizerIntegrationTest extends BaseRequestTest { }, "Expected topic was not created") } + @Test + def testCreatePermissionMetadataRequestAutoCreate() { + val readAcls = topicReadAcl.get(topicResource).get + addAndVerifyAcls(readAcls, topicResource) + assertTrue(zkClient.topicExists(topicResource.name)) + + addAndVerifyAcls(readAcls, createTopicResource) + assertFalse(zkClient.topicExists(createTopic)) + + val metadataRequest = new MetadataRequest.Builder(List(topic, createTopic).asJava, true).build() + val metadataResponse = MetadataResponse.parse(connectAndSend(metadataRequest, ApiKeys.METADATA), ApiKeys.METADATA.latestVersion) + + assertEquals(Set(topic).asJava, metadataResponse.topicsByError(Errors.NONE)); + assertEquals(Set(createTopic).asJava, metadataResponse.topicsByError(Errors.TOPIC_AUTHORIZATION_FAILED)) + + val createAcls = topicCreateAcl.get(createTopicResource).get + addAndVerifyAcls(createAcls, createTopicResource) + + // retry as topic being created can have MetadataResponse with Errors.LEADER_NOT_AVAILABLE + TestUtils.retry(JTestUtils.DEFAULT_MAX_WAIT_MS)(() => { + val metadataResponse = MetadataResponse.parse(connectAndSend(metadataRequest, ApiKeys.METADATA), ApiKeys.METADATA.latestVersion) + assertEquals(Set(topic, createTopic).asJava, metadataResponse.topicsByError(Errors.NONE)) + }) + } + @Test(expected = classOf[AuthorizationException]) def testCommitWithNoAccess() { this.consumers.head.commitSync(Map(tp -> new OffsetAndMetadata(5)).asJava) @@ -973,17 +1003,18 @@ class AuthorizerIntegrationTest extends BaseRequestTest { this.consumers.head.partitionsFor(topic) } - @Test(expected = classOf[GroupAuthorizationException]) + @Test def testDescribeGroupApiWithNoGroupAcl() { addAndVerifyAcls(Set(new Acl(userPrincipal, Allow, Acl.WildCardHost, Describe)), topicResource) - AdminClient.createSimplePlaintext(brokerList).describeConsumerGroup(group) + val result = createAdminClient().describeConsumerGroups(Seq(group).asJava) + TestUtils.assertFutureExceptionTypeEquals(result.describedGroups().get(group), classOf[GroupAuthorizationException]) } @Test def testDescribeGroupApiWithGroupDescribe() { addAndVerifyAcls(Set(new Acl(userPrincipal, Allow, Acl.WildCardHost, Describe)), groupResource) addAndVerifyAcls(Set(new Acl(userPrincipal, Allow, Acl.WildCardHost, Describe)), topicResource) - AdminClient.createSimplePlaintext(brokerList).describeConsumerGroup(group) + createAdminClient().describeConsumerGroups(Seq(group).asJava).describedGroups().get(group).get() } @Test @@ -993,7 +1024,7 @@ class AuthorizerIntegrationTest extends BaseRequestTest { val cgcArgs = Array("--bootstrap-server", brokerList, "--describe", "--group", group) val opts = new ConsumerGroupCommandOptions(cgcArgs) - val consumerGroupService = new KafkaConsumerGroupService(opts) + val consumerGroupService = new ConsumerGroupService(opts) consumerGroupService.describeGroup() consumerGroupService.close() } @@ -1005,8 +1036,7 @@ class AuthorizerIntegrationTest extends BaseRequestTest { addAndVerifyAcls(Set(new Acl(userPrincipal, Allow, Acl.WildCardHost, Delete)), groupResource) this.consumers.head.assign(List(tp).asJava) this.consumers.head.commitSync(Map(tp -> new OffsetAndMetadata(5, "")).asJava) - val result = AdminClient.createSimplePlaintext(brokerList).deleteConsumerGroups(List(group)) - assert(result.size == 1 && result.keySet.contains(group) && result.get(group).contains(Errors.NONE)) + createAdminClient().deleteConsumerGroups(Seq(group).asJava).deletedGroups().get(group).get() } @Test @@ -1015,14 +1045,14 @@ class AuthorizerIntegrationTest extends BaseRequestTest { addAndVerifyAcls(Set(new Acl(userPrincipal, Allow, Acl.WildCardHost, Read)), topicResource) this.consumers.head.assign(List(tp).asJava) this.consumers.head.commitSync(Map(tp -> new OffsetAndMetadata(5, "")).asJava) - val result = AdminClient.createSimplePlaintext(brokerList).deleteConsumerGroups(List(group)) - assert(result.size == 1 && result.keySet.contains(group) && result.get(group).contains(Errors.GROUP_AUTHORIZATION_FAILED)) + val result = createAdminClient().deleteConsumerGroups(Seq(group).asJava) + TestUtils.assertFutureExceptionTypeEquals(result.deletedGroups().get(group), classOf[GroupAuthorizationException]) } @Test def testDeleteGroupApiWithNoDeleteGroupAcl2() { - val result = AdminClient.createSimplePlaintext(brokerList).deleteConsumerGroups(List(group)) - assert(result.size == 1 && result.keySet.contains(group) && result.get(group).contains(Errors.GROUP_AUTHORIZATION_FAILED)) + val result = createAdminClient().deleteConsumerGroups(Seq(group).asJava) + TestUtils.assertFutureExceptionTypeEquals(result.deletedGroups().get(group), classOf[GroupAuthorizationException]) } @Test @@ -1045,7 +1075,7 @@ class AuthorizerIntegrationTest extends BaseRequestTest { @Test def testDeleteTopicsWithWildCardAuth() { - addAndVerifyAcls(Set(new Acl(userPrincipal, Allow, Acl.WildCardHost, Delete)), new Resource(Topic, "*", Literal)) + addAndVerifyAcls(Set(new Acl(userPrincipal, Allow, Acl.WildCardHost, Delete)), Resource(Topic, "*", LITERAL)) val response = connectAndSend(deleteTopicsRequest, ApiKeys.DELETE_TOPICS) val version = ApiKeys.DELETE_TOPICS.latestVersion val deleteResponse = DeleteTopicsResponse.parse(response, version) @@ -1072,7 +1102,7 @@ class AuthorizerIntegrationTest extends BaseRequestTest { @Test def testDeleteRecordsWithWildCardAuth() { - addAndVerifyAcls(Set(new Acl(userPrincipal, Allow, Acl.WildCardHost, Delete)), new Resource(Topic, "*", Literal)) + addAndVerifyAcls(Set(new Acl(userPrincipal, Allow, Acl.WildCardHost, Delete)), Resource(Topic, "*", LITERAL)) val response = connectAndSend(deleteRecordsRequest, ApiKeys.DELETE_RECORDS) val version = ApiKeys.DELETE_RECORDS.latestVersion val deleteRecordsResponse = DeleteRecordsResponse.parse(response, version) @@ -1090,7 +1120,7 @@ class AuthorizerIntegrationTest extends BaseRequestTest { @Test def testCreatePartitionsWithWildCardAuth() { - addAndVerifyAcls(Set(new Acl(userPrincipal, Allow, Acl.WildCardHost, Alter)), new Resource(Topic, "*", Literal)) + addAndVerifyAcls(Set(new Acl(userPrincipal, Allow, Acl.WildCardHost, Alter)), Resource(Topic, "*", LITERAL)) val response = connectAndSend(createPartitionsRequest, ApiKeys.CREATE_PARTITIONS) val version = ApiKeys.CREATE_PARTITIONS.latestVersion val createPartitionsResponse = CreatePartitionsResponse.parse(response, version) @@ -1283,7 +1313,7 @@ class AuthorizerIntegrationTest extends BaseRequestTest { def shouldSuccessfullyAbortTransactionAfterTopicAuthorizationException(): Unit = { addAndVerifyAcls(Set(new Acl(userPrincipal, Allow, Acl.WildCardHost, Write)), transactionalIdResource) addAndVerifyAcls(Set(new Acl(userPrincipal, Allow, Acl.WildCardHost, Write)), topicResource) - addAndVerifyAcls(Set(new Acl(userPrincipal, Allow, Acl.WildCardHost, Describe)), new Resource(Topic, deleteTopic, Literal)) + addAndVerifyAcls(Set(new Acl(userPrincipal, Allow, Acl.WildCardHost, Describe)), Resource(Topic, deleteTopic, LITERAL)) val producer = buildTransactionalProducer() producer.initTransactions() producer.beginTransaction() @@ -1413,7 +1443,7 @@ class AuthorizerIntegrationTest extends BaseRequestTest { private def buildTransactionalProducer(): KafkaProducer[Array[Byte], Array[Byte]] = { val transactionalProperties = new Properties() transactionalProperties.setProperty(ProducerConfig.TRANSACTIONAL_ID_CONFIG, transactionalId) - val producer = TestUtils.createNewProducer(TestUtils.getBrokerListStrFromServers(servers), + val producer = TestUtils.createProducer(TestUtils.getBrokerListStrFromServers(servers), retries = 3, props = Some(transactionalProperties)) producers += producer @@ -1423,11 +1453,19 @@ class AuthorizerIntegrationTest extends BaseRequestTest { private def buildIdempotentProducer(): KafkaProducer[Array[Byte], Array[Byte]] = { val idempotentProperties = new Properties() idempotentProperties.setProperty(ProducerConfig.ENABLE_IDEMPOTENCE_CONFIG, "true") - val producer = TestUtils.createNewProducer(TestUtils.getBrokerListStrFromServers(servers), + val producer = TestUtils.createProducer(TestUtils.getBrokerListStrFromServers(servers), retries = 3, props = Some(idempotentProperties)) producers += producer producer } + private def createAdminClient(): AdminClient = { + val props = new Properties() + props.put(AdminClientConfig.BOOTSTRAP_SERVERS_CONFIG, brokerList) + val adminClient = AdminClient.create(props) + adminClients += adminClient + adminClient + } + } diff --git a/core/src/test/scala/integration/kafka/api/BaseConsumerTest.scala b/core/src/test/scala/integration/kafka/api/BaseConsumerTest.scala index 3521bb6edecbe..59683b82cd908 100644 --- a/core/src/test/scala/integration/kafka/api/BaseConsumerTest.scala +++ b/core/src/test/scala/integration/kafka/api/BaseConsumerTest.scala @@ -31,7 +31,7 @@ import org.apache.kafka.common.errors.WakeupException import org.apache.kafka.common.internals.Topic /** - * Integration tests for the new consumer that cover basic usage as well as server failures + * Integration tests for the consumer that cover basic usage as well as server failures */ abstract class BaseConsumerTest extends IntegrationTestHarness { diff --git a/core/src/test/scala/integration/kafka/api/BaseProducerSendTest.scala b/core/src/test/scala/integration/kafka/api/BaseProducerSendTest.scala index 106984c96f807..3bcf668b42955 100644 --- a/core/src/test/scala/integration/kafka/api/BaseProducerSendTest.scala +++ b/core/src/test/scala/integration/kafka/api/BaseProducerSendTest.scala @@ -21,20 +21,22 @@ import java.nio.charset.StandardCharsets import java.util.Properties import java.util.concurrent.TimeUnit -import collection.JavaConverters._ import kafka.integration.KafkaServerTestHarness import kafka.log.LogConfig import kafka.server.KafkaConfig import kafka.utils.TestUtils import org.apache.kafka.clients.consumer.{ConsumerRecord, KafkaConsumer} import org.apache.kafka.clients.producer._ -import org.apache.kafka.common.{KafkaException, TopicPartition} +import org.apache.kafka.common.errors.TimeoutException import org.apache.kafka.common.record.TimestampType import org.apache.kafka.common.security.auth.SecurityProtocol +import org.apache.kafka.common.{KafkaException, TopicPartition} import org.junit.Assert._ import org.junit.{After, Before, Test} import scala.collection.mutable.{ArrayBuffer, Buffer} +import scala.collection.JavaConverters._ +import scala.concurrent.ExecutionException abstract class BaseProducerSendTest extends KafkaServerTestHarness { @@ -55,7 +57,7 @@ abstract class BaseProducerSendTest extends KafkaServerTestHarness { @Before override def setUp() { super.setUp() - consumer = TestUtils.createNewConsumer(TestUtils.getBrokerListStrFromServers(servers), securityProtocol = SecurityProtocol.PLAINTEXT) + consumer = TestUtils.createConsumer(TestUtils.getBrokerListStrFromServers(servers), securityProtocol = SecurityProtocol.PLAINTEXT) } @After @@ -67,9 +69,19 @@ abstract class BaseProducerSendTest extends KafkaServerTestHarness { super.tearDown() } - protected def createProducer(brokerList: String, retries: Int = 0, lingerMs: Long = 0, props: Option[Properties] = None): KafkaProducer[Array[Byte],Array[Byte]] = { - val producer = TestUtils.createNewProducer(brokerList, securityProtocol = securityProtocol, trustStoreFile = trustStoreFile, - saslProperties = clientSaslProperties, retries = retries, lingerMs = lingerMs, props = props) + protected def createProducer(brokerList: String, + retries: Int = 0, + lingerMs: Long = 0, + maxBlockMs: Long = 60 * 1000L, + props: Option[Properties] = None): KafkaProducer[Array[Byte],Array[Byte]] = { + val producer = TestUtils.createProducer(brokerList, + securityProtocol = securityProtocol, + trustStoreFile = trustStoreFile, + saslProperties = clientSaslProperties, + retries = retries, + lingerMs = lingerMs, + maxBlockMs = maxBlockMs, + props = props) registerProducer(producer) } @@ -345,7 +357,7 @@ abstract class BaseProducerSendTest extends KafkaServerTestHarness { */ @Test def testSendBeforeAndAfterPartitionExpansion() { - val producer = createProducer(brokerList) + val producer = createProducer(brokerList, maxBlockMs = 5 * 1000L) // create topic createTopic(topic, 1, 2) @@ -365,10 +377,13 @@ abstract class BaseProducerSendTest extends KafkaServerTestHarness { // Trying to send a record to a partition beyond topic's partition range before adding the partition should fail. val partition1 = 1 try { - producer.send(new ProducerRecord(topic, partition1, null, "value".getBytes(StandardCharsets.UTF_8))) + producer.send(new ProducerRecord(topic, partition1, null, "value".getBytes(StandardCharsets.UTF_8))).get() fail("Should not allow sending a record to a partition not present in the metadata") } catch { - case _: KafkaException => // this is ok + case e: ExecutionException => e.getCause match { + case _: TimeoutException => // this is ok + case ex => throw new Exception("Sending to a partition not present in the metadata should result in a TimeoutException", ex) + } } val existingAssignment = zkClient.getReplicaAssignmentForTopics(Set(topic)).map { @@ -446,8 +461,7 @@ abstract class BaseProducerSendTest extends KafkaServerTestHarness { future.get() fail("No message should be sent successfully.") } catch { - case e: Exception => - assertEquals("java.lang.IllegalStateException: Producer is closed forcefully.", e.getMessage) + case e: ExecutionException => assertEquals(classOf[KafkaException], e.getCause.getClass) } } assertEquals("Fetch response should have no message returned.", 0, consumer.poll(50).count) diff --git a/core/src/test/scala/integration/kafka/api/ConsumerBounceTest.scala b/core/src/test/scala/integration/kafka/api/ConsumerBounceTest.scala index 53b3ed679bb0c..07cbf0cf414dd 100644 --- a/core/src/test/scala/integration/kafka/api/ConsumerBounceTest.scala +++ b/core/src/test/scala/integration/kafka/api/ConsumerBounceTest.scala @@ -14,28 +14,35 @@ package kafka.api import java.util.concurrent._ -import java.util.{Collection, Collections} +import java.util.{Collection, Collections, Properties} -import kafka.admin.AdminClient -import kafka.server.KafkaConfig +import kafka.server.{BaseRequestTest, KafkaConfig} import kafka.utils.{CoreUtils, Logging, ShutdownableThread, TestUtils} import org.apache.kafka.clients.consumer._ -import org.apache.kafka.clients.producer.{ProducerConfig, ProducerRecord} +import org.apache.kafka.clients.producer.{KafkaProducer, ProducerConfig, ProducerRecord} import org.apache.kafka.common.TopicPartition +import org.apache.kafka.common.protocol.ApiKeys +import org.apache.kafka.common.requests.{FindCoordinatorRequest, FindCoordinatorResponse} +import org.apache.kafka.common.security.auth.SecurityProtocol import org.junit.Assert._ import org.junit.{After, Before, Ignore, Test} import scala.collection.JavaConverters._ +import scala.collection.mutable.Buffer /** - * Integration tests for the new consumer that cover basic usage as well as server failures + * Integration tests for the consumer that cover basic usage as well as server failures */ -class ConsumerBounceTest extends IntegrationTestHarness with Logging { +class ConsumerBounceTest extends BaseRequestTest with Logging { + + override def numBrokers: Int = 3 val producerCount = 1 val consumerCount = 2 - val serverCount = 3 + + val consumers = Buffer[KafkaConsumer[Array[Byte], Array[Byte]]]() + val producers = Buffer[KafkaProducer[Array[Byte], Array[Byte]]]() val topic = "topic" val part = 0 @@ -45,13 +52,8 @@ class ConsumerBounceTest extends IntegrationTestHarness with Logging { val gracefulCloseTimeMs = 1000 val executor = Executors.newScheduledThreadPool(2) - // configure the servers and clients - this.serverConfig.setProperty(KafkaConfig.OffsetsTopicReplicationFactorProp, "3") // don't want to lose offset - this.serverConfig.setProperty(KafkaConfig.OffsetsTopicPartitionsProp, "1") - this.serverConfig.setProperty(KafkaConfig.GroupMinSessionTimeoutMsProp, "10") // set small enough session timeout - this.serverConfig.setProperty(KafkaConfig.GroupInitialRebalanceDelayMsProp, "0") - this.serverConfig.setProperty(KafkaConfig.UncleanLeaderElectionEnableProp, "true") - this.serverConfig.setProperty(KafkaConfig.AutoCreateTopicsEnableProp, "false") + val producerConfig = new Properties + val consumerConfig = new Properties 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) @@ -59,8 +61,19 @@ class ConsumerBounceTest extends IntegrationTestHarness with Logging { this.consumerConfig.setProperty(ConsumerConfig.HEARTBEAT_INTERVAL_MS_CONFIG, "3000") this.consumerConfig.setProperty(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG, "earliest") + def serverConfig(): Properties = { + val properties = new Properties + properties.put(KafkaConfig.OffsetsTopicReplicationFactorProp, "3") // don't want to lose offset + properties.put(KafkaConfig.OffsetsTopicPartitionsProp, "1") + properties.put(KafkaConfig.GroupMinSessionTimeoutMsProp, "10") // set small enough session timeout + properties.put(KafkaConfig.GroupInitialRebalanceDelayMsProp, "0") + properties.put(KafkaConfig.UncleanLeaderElectionEnableProp, "true") + properties.put(KafkaConfig.AutoCreateTopicsEnableProp, "false") + properties + } + override def generateConfigs = { - FixedPortTestUtils.createBrokerConfigs(serverCount, zkConnect, enableControlledShutdown = false) + FixedPortTestUtils.createBrokerConfigs(numBrokers, zkConnect, enableControlledShutdown = false) .map(KafkaConfig.fromProps(_, serverConfig)) } @@ -68,8 +81,26 @@ class ConsumerBounceTest extends IntegrationTestHarness with Logging { override def setUp() { super.setUp() + for (_ <- 0 until producerCount) + producers += createProducer + + for (_ <- 0 until consumerCount) + consumers += createConsumer + // create the test topic with all the brokers as replicas - createTopic(topic, 1, serverCount) + createTopic(topic, 1, numBrokers) + } + + def createProducer: KafkaProducer[Array[Byte], Array[Byte]] = { + TestUtils.createProducer(TestUtils.getBrokerListStrFromServers(servers), + securityProtocol = SecurityProtocol.PLAINTEXT, + props = Some(producerConfig)) + } + + def createConsumer: KafkaConsumer[Array[Byte], Array[Byte]] = { + TestUtils.createConsumer(TestUtils.getBrokerListStrFromServers(servers), + securityProtocol = SecurityProtocol.PLAINTEXT, + props = Some(consumerConfig)) } @After @@ -78,6 +109,8 @@ class ConsumerBounceTest extends IntegrationTestHarness with Logging { executor.shutdownNow() // Wait for any active tasks to terminate to ensure consumer is not closed while being used from another thread assertTrue("Executor did not terminate", executor.awaitTermination(5000, TimeUnit.MILLISECONDS)) + producers.foreach(_.close()) + consumers.foreach(_.close()) } finally { super.tearDown() } @@ -173,7 +206,7 @@ class ConsumerBounceTest extends IntegrationTestHarness with Logging { val consumer = this.consumers.head consumer.subscribe(Collections.singleton(newtopic)) executor.schedule(new Runnable { - def run() = createTopic(newtopic, numPartitions = serverCount, replicationFactor = serverCount) + def run() = createTopic(newtopic, numPartitions = numBrokers, replicationFactor = numBrokers) }, 2, TimeUnit.SECONDS) consumer.poll(0) @@ -243,9 +276,8 @@ class ConsumerBounceTest extends IntegrationTestHarness with Logging { val consumer1 = createConsumerAndReceive(dynamicGroup, false, numRecords) val consumer2 = createConsumerAndReceive(manualGroup, true, numRecords) - val adminClient = AdminClient.createSimplePlaintext(this.brokerList) - killBroker(adminClient.findCoordinator(dynamicGroup).id) - killBroker(adminClient.findCoordinator(manualGroup).id) + killBroker(findCoordinator(dynamicGroup)) + killBroker(findCoordinator(manualGroup)) val future1 = submitCloseAndValidate(consumer1, Long.MaxValue, None, Some(gracefulCloseTimeMs)) val future2 = submitCloseAndValidate(consumer2, Long.MaxValue, None, Some(gracefulCloseTimeMs)) @@ -255,9 +287,16 @@ class ConsumerBounceTest extends IntegrationTestHarness with Logging { restartDeadBrokers() checkClosedState(dynamicGroup, 0) checkClosedState(manualGroup, numRecords) - adminClient.close() } + private def findCoordinator(group: String) : Int = { + val request = new FindCoordinatorRequest.Builder(FindCoordinatorRequest.CoordinatorType.GROUP, group).build() + val resp = connectAndSend(request, ApiKeys.FIND_COORDINATOR) + val response = FindCoordinatorResponse.parse(resp, ApiKeys.FIND_COORDINATOR.latestVersion()) + response.node().id() + } + + /** * Consumer is closed while all brokers are unavailable. Cannot rebalance or commit offsets since * there is no coordinator, but close should timeout and return. If close is invoked with a very @@ -288,7 +327,7 @@ class ConsumerBounceTest extends IntegrationTestHarness with Logging { @Test def testCloseDuringRebalance() { val topic = "closetest" - createTopic(topic, 10, serverCount) + createTopic(topic, 10, numBrokers) this.consumerConfig.setProperty(ConsumerConfig.MAX_POLL_INTERVAL_MS_CONFIG, "60000") this.consumerConfig.setProperty(ConsumerConfig.HEARTBEAT_INTERVAL_MS_CONFIG, "1000") this.consumerConfig.setProperty(ConsumerConfig.ENABLE_AUTO_COMMIT_CONFIG, "false") @@ -355,7 +394,7 @@ class ConsumerBounceTest extends IntegrationTestHarness with Logging { private def createConsumer(groupId: String) : KafkaConsumer[Array[Byte], Array[Byte]] = { this.consumerConfig.setProperty(ConsumerConfig.GROUP_ID_CONFIG, groupId) - val consumer = createNewConsumer + val consumer = createConsumer consumers += consumer consumer } diff --git a/core/src/test/scala/integration/kafka/api/CustomQuotaCallbackTest.scala b/core/src/test/scala/integration/kafka/api/CustomQuotaCallbackTest.scala index ffe7ffdcd5e6d..eb8f11d21cf1b 100644 --- a/core/src/test/scala/integration/kafka/api/CustomQuotaCallbackTest.scala +++ b/core/src/test/scala/integration/kafka/api/CustomQuotaCallbackTest.scala @@ -77,7 +77,7 @@ class CustomQuotaCallbackTest extends IntegrationTestHarness with SaslSetup { producerConfig.put(SaslConfigs.SASL_JAAS_CONFIG, ScramLoginModule(JaasTestUtils.KafkaScramAdmin, JaasTestUtils.KafkaScramAdminPassword).toString) - producerWithoutQuota = createNewProducer + producerWithoutQuota = createProducer producers += producerWithoutQuota } @@ -89,6 +89,7 @@ class CustomQuotaCallbackTest extends IntegrationTestHarness with SaslSetup { producers.clear() consumers.foreach(_.close(0, TimeUnit.MILLISECONDS)) consumers.clear() + adminClients.foreach(_.close()) super.tearDown() } @@ -225,13 +226,13 @@ class CustomQuotaCallbackTest extends IntegrationTestHarness with SaslSetup { producerConfig.put(ProducerConfig.CLIENT_ID_CONFIG, producerClientId) producerConfig.put(SaslConfigs.SASL_JAAS_CONFIG, ScramLoginModule(user, password).toString) - val producer = createNewProducer + val producer = createProducer producers += producer consumerConfig.put(ConsumerConfig.CLIENT_ID_CONFIG, consumerClientId) consumerConfig.put(ConsumerConfig.GROUP_ID_CONFIG, s"$user-group") consumerConfig.put(SaslConfigs.SASL_JAAS_CONFIG, ScramLoginModule(user, password).toString) - val consumer = createNewConsumer + val consumer = createConsumer consumers += consumer GroupedUser(user, userGroup, topic, servers(leader), producerClientId, consumerClientId, producer, consumer) diff --git a/core/src/test/scala/integration/kafka/api/EndToEndAuthorizationTest.scala b/core/src/test/scala/integration/kafka/api/EndToEndAuthorizationTest.scala index b809686f80585..b126486e8c1a8 100644 --- a/core/src/test/scala/integration/kafka/api/EndToEndAuthorizationTest.scala +++ b/core/src/test/scala/integration/kafka/api/EndToEndAuthorizationTest.scala @@ -31,6 +31,7 @@ import org.apache.kafka.clients.producer.{KafkaProducer, ProducerRecord} import org.apache.kafka.common.security.auth.KafkaPrincipal import org.apache.kafka.common.{KafkaException, TopicPartition} import org.apache.kafka.common.errors.{GroupAuthorizationException, TimeoutException, TopicAuthorizationException} +import org.apache.kafka.common.resource.PatternType.{LITERAL, PREFIXED} import org.junit.Assert._ import org.junit.{After, Before, Test} @@ -78,13 +79,13 @@ abstract class EndToEndAuthorizationTest extends IntegrationTestHarness with Sas override protected lazy val trustStoreFile = Some(File.createTempFile("truststore", ".jks")) - val topicResource = new Resource(Topic, topic, Literal) - val groupResource = new Resource(Group, group, Literal) + val topicResource = Resource(Topic, topic, LITERAL) + val groupResource = Resource(Group, group, LITERAL) val clusterResource = Resource.ClusterResource - val prefixedTopicResource = new Resource(Topic, topicPrefix, Prefixed) - val prefixedGroupResource = new Resource(Group, groupPrefix, Prefixed) - val wildcardTopicResource = new Resource(Topic, wildcard, Literal) - val wildcardGroupResource = new Resource(Group, wildcard, Literal) + val prefixedTopicResource = Resource(Topic, topicPrefix, PREFIXED) + val prefixedGroupResource = Resource(Group, groupPrefix, PREFIXED) + val wildcardTopicResource = Resource(Topic, wildcard, LITERAL) + val wildcardGroupResource = Resource(Group, wildcard, LITERAL) // Arguments to AclCommand to set ACLs. def clusterActionArgs: Array[String] = Array("--authorizer-properties", @@ -151,7 +152,7 @@ abstract class EndToEndAuthorizationTest extends IntegrationTestHarness with Sas s"--add", s"--topic=$topicPrefix", s"--group=$groupPrefix", - s"--resource-name-type=prefixed", + s"--resource-pattern-type=prefixed", s"--consumer", s"--producer", s"--allow-principal=$kafkaPrincipalType:$clientPrincipal") @@ -182,14 +183,14 @@ abstract class EndToEndAuthorizationTest extends IntegrationTestHarness with Sas super.setUp() servers.foreach { s => TestUtils.waitAndVerifyAcls(ClusterActionAcl, s.apis.authorizer.get, Resource.ClusterResource) - TestUtils.waitAndVerifyAcls(TopicBrokerReadAcl, s.apis.authorizer.get, new Resource(Topic, "*", Literal)) + TestUtils.waitAndVerifyAcls(TopicBrokerReadAcl, s.apis.authorizer.get, Resource(Topic, "*", LITERAL)) } // create the test topic with all the brokers as replicas createTopic(topic, 1, 3) } - override def createNewProducer: KafkaProducer[Array[Byte], Array[Byte]] = { - TestUtils.createNewProducer(brokerList, + override def createProducer: KafkaProducer[Array[Byte], Array[Byte]] = { + TestUtils.createProducer(brokerList, maxBlockMs = 3000L, securityProtocol = this.securityProtocol, trustStoreFile = this.trustStoreFile, diff --git a/core/src/test/scala/integration/kafka/api/GroupCoordinatorIntegrationTest.scala b/core/src/test/scala/integration/kafka/api/GroupCoordinatorIntegrationTest.scala index 2049e0a6a8ff2..b0e9570020a3d 100644 --- a/core/src/test/scala/integration/kafka/api/GroupCoordinatorIntegrationTest.scala +++ b/core/src/test/scala/integration/kafka/api/GroupCoordinatorIntegrationTest.scala @@ -40,7 +40,7 @@ class GroupCoordinatorIntegrationTest extends KafkaServerTestHarness { @Test def testGroupCoordinatorPropagatesOfffsetsTopicCompressionCodec() { - val consumer = TestUtils.createNewConsumer(TestUtils.getBrokerListStrFromServers(servers), + val consumer = TestUtils.createConsumer(TestUtils.getBrokerListStrFromServers(servers), securityProtocol = SecurityProtocol.PLAINTEXT) val offsetMap = Map( new TopicPartition(Topic.GROUP_METADATA_TOPIC_NAME, 0) -> new OffsetAndMetadata(10, "") diff --git a/core/src/test/scala/integration/kafka/api/IntegrationTestHarness.scala b/core/src/test/scala/integration/kafka/api/IntegrationTestHarness.scala index 1b45acbae82c7..053f04e0cdc56 100644 --- a/core/src/test/scala/integration/kafka/api/IntegrationTestHarness.scala +++ b/core/src/test/scala/integration/kafka/api/IntegrationTestHarness.scala @@ -79,9 +79,9 @@ abstract class IntegrationTestHarness extends KafkaServerTestHarness { consumerConfig.put(ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG, classOf[org.apache.kafka.common.serialization.ByteArrayDeserializer]) consumerConfig ++= consumerSecurityProps for (_ <- 0 until producerCount) - producers += createNewProducer + producers += createProducer for (_ <- 0 until consumerCount) { - consumers += createNewConsumer + consumers += createConsumer } TestUtils.createOffsetsTopic(zkClient, servers) @@ -92,16 +92,16 @@ abstract class IntegrationTestHarness extends KafkaServerTestHarness { clientSaslProperties) } - def createNewProducer: KafkaProducer[Array[Byte], Array[Byte]] = { - TestUtils.createNewProducer(brokerList, + def createProducer: KafkaProducer[Array[Byte], Array[Byte]] = { + TestUtils.createProducer(brokerList, securityProtocol = this.securityProtocol, trustStoreFile = this.trustStoreFile, saslProperties = this.clientSaslProperties, props = Some(producerConfig)) } - def createNewConsumer: KafkaConsumer[Array[Byte], Array[Byte]] = { - TestUtils.createNewConsumer(brokerList, + def createConsumer: KafkaConsumer[Array[Byte], Array[Byte]] = { + TestUtils.createConsumer(brokerList, securityProtocol = this.securityProtocol, trustStoreFile = this.trustStoreFile, saslProperties = this.clientSaslProperties, diff --git a/core/src/test/scala/integration/kafka/api/LegacyAdminClientTest.scala b/core/src/test/scala/integration/kafka/api/LegacyAdminClientTest.scala index b78946cde3025..115ec0539c309 100644 --- a/core/src/test/scala/integration/kafka/api/LegacyAdminClientTest.scala +++ b/core/src/test/scala/integration/kafka/api/LegacyAdminClientTest.scala @@ -24,7 +24,7 @@ import java.lang.{Long => JLong} import kafka.utils.{Logging, TestUtils} import org.apache.kafka.clients.consumer.{ConsumerConfig, KafkaConsumer} -import org.apache.kafka.clients.producer.{KafkaProducer, ProducerConfig, ProducerRecord} +import org.apache.kafka.clients.producer.ProducerConfig import org.apache.kafka.common.TopicPartition import org.apache.kafka.common.protocol.ApiKeys import org.junit.{After, Before, Test} @@ -147,16 +147,4 @@ class LegacyAdminClientTest extends IntegrationTestHarness with Logging { }, "Expected non-empty assignment") } - private def sendRecords(producer: KafkaProducer[Array[Byte], Array[Byte]], - numRecords: Int, - tp: TopicPartition) { - val futures = (0 until numRecords).map { i => - val record = new ProducerRecord(tp.topic(), tp.partition(), s"$i".getBytes, s"$i".getBytes) - debug(s"Sending this record: $record") - producer.send(record) - } - - futures.foreach(_.get) - } - } diff --git a/core/src/test/scala/integration/kafka/api/MetricsTest.scala b/core/src/test/scala/integration/kafka/api/MetricsTest.scala index cea3d279d62be..fe7563971ee7f 100644 --- a/core/src/test/scala/integration/kafka/api/MetricsTest.scala +++ b/core/src/test/scala/integration/kafka/api/MetricsTest.scala @@ -119,7 +119,7 @@ class MetricsTest extends IntegrationTestHarness with SaslSetup { saslProps.put(SaslConfigs.SASL_MECHANISM, "SCRAM-SHA-256") // Use acks=0 to verify error metric when connection is closed without a response saslProps.put(ProducerConfig.ACKS_CONFIG, "0") - val producer = TestUtils.createNewProducer(brokerList, securityProtocol = securityProtocol, + val producer = TestUtils.createProducer(brokerList, securityProtocol = securityProtocol, trustStoreFile = trustStoreFile, saslProperties = Some(saslProps), props = Some(producerProps)) try { diff --git a/core/src/test/scala/integration/kafka/api/PlaintextConsumerTest.scala b/core/src/test/scala/integration/kafka/api/PlaintextConsumerTest.scala index 372cc3ffed694..02396dd1bb24d 100644 --- a/core/src/test/scala/integration/kafka/api/PlaintextConsumerTest.scala +++ b/core/src/test/scala/integration/kafka/api/PlaintextConsumerTest.scala @@ -43,11 +43,11 @@ class PlaintextConsumerTest extends BaseConsumerTest { def testHeaders() { val numRecords = 1 val record = new ProducerRecord(tp.topic, tp.partition, null, "key".getBytes, "value".getBytes) - + record.headers().add("headerKey", "headerValue".getBytes) - + this.producers.head.send(record) - + assertEquals(0, this.consumers.head.assignment.size) this.consumers.head.assign(List(tp).asJava) assertEquals(1, this.consumers.head.assignment.size) @@ -63,23 +63,23 @@ class PlaintextConsumerTest extends BaseConsumerTest { assertEquals("headerValue", if (header == null) null else new String(header.value())) } } - + @Test def testHeadersExtendedSerializerDeserializer() { val numRecords = 1 val record = new ProducerRecord(tp.topic, tp.partition, null, "key".getBytes, "value".getBytes) val extendedSerializer = new ExtendedSerializer[Array[Byte]] { - + var serializer = new ByteArraySerializer() - + override def serialize(topic: String, headers: Headers, data: Array[Byte]): Array[Byte] = { headers.add("content-type", "application/octet-stream".getBytes) serializer.serialize(topic, data) } override def configure(configs: util.Map[String, _], isKey: Boolean): Unit = serializer.configure(configs, isKey) - + override def close(): Unit = serializer.close() override def serialize(topic: String, data: Array[Byte]): Array[Byte] = { @@ -90,9 +90,9 @@ class PlaintextConsumerTest extends BaseConsumerTest { val extendedDeserializer = new ExtendedDeserializer[Array[Byte]] { - + var deserializer = new ByteArrayDeserializer() - + override def deserialize(topic: String, headers: Headers, data: Array[Byte]): Array[Byte] = { val header = headers.lastHeader("content-type") assertEquals("application/octet-stream", if (header == null) null else new String(header.value())) @@ -110,7 +110,7 @@ class PlaintextConsumerTest extends BaseConsumerTest { } } - + val producer0 = new KafkaProducer(this.producerConfig, new ByteArraySerializer(), extendedSerializer) producers += producer0 producer0.send(record) @@ -127,7 +127,7 @@ class PlaintextConsumerTest extends BaseConsumerTest { assertEquals(numRecords, records.size) } - + @Test def testMaxPollRecords() { val maxPollRecords = 2 @@ -618,7 +618,7 @@ class PlaintextConsumerTest extends BaseConsumerTest { val producerProps = new Properties() producerProps.setProperty(ProducerConfig.COMPRESSION_TYPE_CONFIG, CompressionType.GZIP.name) producerProps.setProperty(ProducerConfig.LINGER_MS_CONFIG, Long.MaxValue.toString) - val producer = TestUtils.createNewProducer(brokerList, securityProtocol = securityProtocol, trustStoreFile = trustStoreFile, + val producer = TestUtils.createProducer(brokerList, securityProtocol = securityProtocol, trustStoreFile = trustStoreFile, saslProperties = clientSaslProperties, retries = 0, lingerMs = Long.MaxValue, props = Some(producerProps)) (0 until numRecords).foreach { i => producer.send(new ProducerRecord(tp.topic, tp.partition, i.toLong, s"key $i".getBytes, s"value $i".getBytes)) @@ -1534,10 +1534,6 @@ class PlaintextConsumerTest extends BaseConsumerTest { val fetchLag = consumer.metrics.get(new MetricName("records-lag", "consumer-fetch-manager-metrics", "", tags)) assertNotNull(fetchLag) - val oldTags = Collections.singletonMap("client-id", "testPerPartitionLagMetricsCleanUpWithAssign") - val oldFetchLag = consumer.metrics.get(new MetricName(tp + ".records-lag", "consumer-fetch-manager-metrics", "", oldTags)) - assertEquals(fetchLag.metricValue(), oldFetchLag.metricValue()) - val expectedLag = numMessages - records.count assertEquals(s"The lag should be $expectedLag", expectedLag, fetchLag.value, epsilon) @@ -1594,15 +1590,12 @@ class PlaintextConsumerTest extends BaseConsumerTest { records = consumer.poll(100) !records.isEmpty }, "Consumer did not consume any message before timeout.") - val oldTags = Collections.singletonMap("client-id", "testPerPartitionLagWithMaxPollRecords") - val oldLag = consumer.metrics.get(new MetricName(tp + ".records-lag", "consumer-fetch-manager-metrics", "", oldTags)) val tags = new util.HashMap[String, String]() tags.put("client-id", "testPerPartitionLagWithMaxPollRecords") tags.put("topic", tp.topic()) tags.put("partition", String.valueOf(tp.partition())) val lag = consumer.metrics.get(new MetricName("records-lag", "consumer-fetch-manager-metrics", "", tags)) - assertEquals(oldLag.metricValue(), lag.metricValue()) assertEquals(s"The lag should be ${numMessages - records.count}", numMessages - records.count, lag.value, epsilon) } finally { @@ -1778,9 +1771,9 @@ class PlaintextConsumerTest extends BaseConsumerTest { subscriptions: Set[TopicPartition]): Unit = { assertTrue(consumerGroup.size + numOfConsumersToAdd <= subscriptions.size) for (_ <- 0 until numOfConsumersToAdd) { - val newConsumer = new KafkaConsumer[Array[Byte], Array[Byte]](this.consumerConfig) - consumerGroup += newConsumer - consumerPollers += subscribeConsumerAndStartPolling(newConsumer, topicsToSubscribe) + val consumer = new KafkaConsumer[Array[Byte], Array[Byte]](this.consumerConfig) + consumerGroup += consumer + consumerPollers += subscribeConsumerAndStartPolling(consumer, topicsToSubscribe) } // wait until topics get re-assigned and validate assignment diff --git a/core/src/test/scala/integration/kafka/api/ProducerBounceTest.scala b/core/src/test/scala/integration/kafka/api/ProducerBounceTest.scala deleted file mode 100644 index e3514cd785deb..0000000000000 --- a/core/src/test/scala/integration/kafka/api/ProducerBounceTest.scala +++ /dev/null @@ -1,170 +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.util.Properties -import java.util.concurrent.Future - -import kafka.consumer.SimpleConsumer -import kafka.integration.KafkaServerTestHarness -import kafka.server.KafkaConfig -import kafka.utils.{ShutdownableThread, TestUtils} -import kafka.utils.Implicits._ -import org.apache.kafka.clients.producer._ -import org.apache.kafka.clients.producer.internals.ErrorLoggingCallback -import org.junit.Assert._ -import org.junit.{Ignore, Test} - -import scala.collection.mutable.ArrayBuffer - -class ProducerBounceTest extends KafkaServerTestHarness { - private val producerBufferSize = 65536 - private val serverMessageMaxBytes = producerBufferSize/2 - - val numServers = 4 - - 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) - overridingProps.put(KafkaConfig.ControlledShutdownEnableProp, true.toString) - overridingProps.put(KafkaConfig.UncleanLeaderElectionEnableProp, false.toString) - overridingProps.put(KafkaConfig.AutoLeaderRebalanceEnableProp, false.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 = true) - .map(KafkaConfig.fromProps(_, overridingProps)) - } - - private val topic1 = "topic-1" - - /** - * With replication, producer should able to find new leader after it detects broker failure - */ - @Ignore // To be re-enabled once we can make it less flaky (KAFKA-2837) - @Test - def testBrokerFailure() { - val numPartitions = 3 - val topicConfig = new Properties() - topicConfig.put(KafkaConfig.MinInSyncReplicasProp, 2.toString) - createTopic(topic1, numPartitions, numServers, topicConfig) - - val scheduler = new ProducerScheduler() - scheduler.start() - - // rolling bounce brokers - - for (_ <- 0 until numServers) { - for (server <- servers) { - info("Shutting down server : %s".format(server.config.brokerId)) - server.shutdown() - server.awaitShutdown() - info("Server %s shut down. Starting it up again.".format(server.config.brokerId)) - server.startup() - info("Restarted server: %s".format(server.config.brokerId)) - } - - // Make sure the producer do not see any exception in returned metadata due to broker failures - assertFalse(scheduler.failed) - - // 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 - assertFalse(scheduler.failed) - - // 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", boundPort(servers(leader)), 30000, 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 - info(s"number of unique messages sent: ${uniqueMessageSize}") - assertEquals(s"Found ${messages.size - uniqueMessageSize} duplicate messages.", uniqueMessageSize, messages.size) - assertEquals("Should have fetched " + scheduler.sent + " unique messages", scheduler.sent, messages.size) - } - - private class ProducerScheduler extends ShutdownableThread("daemon-producer", false) { - val numRecords = 1000 - var sent = 0 - var failed = false - - val producerConfig = new Properties() - producerConfig.put(ProducerConfig.ENABLE_IDEMPOTENCE_CONFIG, "true") - producerConfig.put(ProducerConfig.MAX_IN_FLIGHT_REQUESTS_PER_CONNECTION, "5") - val producerConfigWithCompression = new Properties() - producerConfigWithCompression ++= producerConfig - producerConfigWithCompression.put(ProducerConfig.COMPRESSION_TYPE_CONFIG, "lz4") - val producers = List( - TestUtils.createNewProducer(brokerList, bufferSize = producerBufferSize / 4, retries = 10, props = Some(producerConfig)), - TestUtils.createNewProducer(brokerList, bufferSize = producerBufferSize / 2, retries = 10, lingerMs = 5000, props = Some(producerConfig)), - TestUtils.createNewProducer(brokerList, bufferSize = producerBufferSize, retries = 10, lingerMs = 10000, props = Some(producerConfigWithCompression)) - ) - - override def doWork(): Unit = { - info("Starting to send messages..") - var producerId = 0 - val responses = new ArrayBuffer[IndexedSeq[Future[RecordMetadata]]]() - for (producer <- producers) { - val response = - for (i <- sent+1 to sent+numRecords) - yield producer.send(new ProducerRecord[Array[Byte],Array[Byte]](topic1, null, null, ((producerId + 1) * i).toString.getBytes), - new ErrorLoggingCallback(topic1, null, null, true)) - responses.append(response) - producerId += 1 - } - - try { - for (response <- responses) { - val futures = response.toList - futures.map(_.get) - sent += numRecords - } - info(s"Sent $sent records") - } catch { - case e : Exception => - error(s"Got exception ${e.getMessage}") - e.printStackTrace() - failed = true - } - } - - override def shutdown(){ - super.shutdown() - for (producer <- producers) { - producer.close() - } - } - } -} diff --git a/core/src/test/scala/integration/kafka/api/ProducerCompressionTest.scala b/core/src/test/scala/integration/kafka/api/ProducerCompressionTest.scala index 3fe86d440851e..e4a18284aa9fa 100755 --- a/core/src/test/scala/integration/kafka/api/ProducerCompressionTest.scala +++ b/core/src/test/scala/integration/kafka/api/ProducerCompressionTest.scala @@ -70,7 +70,7 @@ class ProducerCompressionTest(compression: String) extends ZooKeeperTestHarness producerProps.put(ProducerConfig.BATCH_SIZE_CONFIG, "66000") producerProps.put(ProducerConfig.LINGER_MS_CONFIG, "200") val producer = new KafkaProducer(producerProps, new ByteArraySerializer, new ByteArraySerializer) - val consumer = TestUtils.createNewConsumer(bootstrapServers, securityProtocol = SecurityProtocol.PLAINTEXT) + val consumer = TestUtils.createConsumer(bootstrapServers, securityProtocol = SecurityProtocol.PLAINTEXT) try { // create topic diff --git a/core/src/test/scala/integration/kafka/api/ProducerFailureHandlingTest.scala b/core/src/test/scala/integration/kafka/api/ProducerFailureHandlingTest.scala index 8ca5163af7fd8..6fda4d19888b4 100644 --- a/core/src/test/scala/integration/kafka/api/ProducerFailureHandlingTest.scala +++ b/core/src/test/scala/integration/kafka/api/ProducerFailureHandlingTest.scala @@ -17,7 +17,7 @@ package kafka.api -import java.util.concurrent.{ExecutionException, TimeoutException} +import java.util.concurrent.ExecutionException import java.util.Properties import kafka.integration.KafkaServerTestHarness @@ -25,7 +25,6 @@ import kafka.log.LogConfig import kafka.server.KafkaConfig import kafka.utils.TestUtils import org.apache.kafka.clients.producer._ -import org.apache.kafka.common.KafkaException import org.apache.kafka.common.errors._ import org.apache.kafka.common.internals.Topic import org.apache.kafka.common.record.{DefaultRecord, DefaultRecordBatch} @@ -64,11 +63,11 @@ class ProducerFailureHandlingTest extends KafkaServerTestHarness { override def setUp() { super.setUp() - producer1 = TestUtils.createNewProducer(brokerList, acks = 0, requestTimeoutMs = 30000L, maxBlockMs = 10000L, + producer1 = TestUtils.createProducer(brokerList, acks = 0, requestTimeoutMs = 30000L, maxBlockMs = 10000L, bufferSize = producerBufferSize) - producer2 = TestUtils.createNewProducer(brokerList, acks = 1, requestTimeoutMs = 30000L, maxBlockMs = 10000L, + producer2 = TestUtils.createProducer(brokerList, acks = 1, requestTimeoutMs = 30000L, maxBlockMs = 10000L, bufferSize = producerBufferSize) - producer3 = TestUtils.createNewProducer(brokerList, acks = -1, requestTimeoutMs = 30000L, maxBlockMs = 10000L, + producer3 = TestUtils.createProducer(brokerList, acks = -1, requestTimeoutMs = 30000L, maxBlockMs = 10000L, bufferSize = producerBufferSize) } @@ -172,7 +171,7 @@ class ProducerFailureHandlingTest extends KafkaServerTestHarness { createTopic(topic1, replicationFactor = numServers) // producer with incorrect broker list - producer4 = TestUtils.createNewProducer("localhost:8686,localhost:4242", acks = 1, maxBlockMs = 10000L, bufferSize = producerBufferSize) + producer4 = TestUtils.createProducer("localhost:8686,localhost:4242", acks = 1, maxBlockMs = 10000L, bufferSize = producerBufferSize) // send a record with incorrect broker list val record = new ProducerRecord(topic1, null, "key".getBytes, "value".getBytes) @@ -182,8 +181,8 @@ class ProducerFailureHandlingTest extends KafkaServerTestHarness { } /** - * Send with invalid partition id should throw KafkaException when partition is higher than the upper bound of - * partitions. + * Send with invalid partition id should return ExecutionException caused by TimeoutException + * when partition is higher than the upper bound of partitions. */ @Test def testInvalidPartition() { @@ -192,8 +191,11 @@ class ProducerFailureHandlingTest extends KafkaServerTestHarness { // create a record with incorrect partition id (higher than the number of partitions), send should fail val higherRecord = new ProducerRecord(topic1, 1, "key".getBytes, "value".getBytes) - intercept[KafkaException] { - producer1.send(higherRecord) + intercept[ExecutionException] { + producer1.send(higherRecord).get + }.getCause match { + case _: TimeoutException => // this is ok + case ex => throw new Exception("Sending to a partition not present in the metadata should result in a TimeoutException", ex) } } @@ -205,7 +207,7 @@ class ProducerFailureHandlingTest extends KafkaServerTestHarness { // create topic createTopic(topic1, replicationFactor = numServers) - val record = new ProducerRecord[Array[Byte],Array[Byte]](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 diff --git a/core/src/test/scala/integration/kafka/api/RackAwareAutoTopicCreationTest.scala b/core/src/test/scala/integration/kafka/api/RackAwareAutoTopicCreationTest.scala index 197214e64441b..bfd2924307d5c 100644 --- a/core/src/test/scala/integration/kafka/api/RackAwareAutoTopicCreationTest.scala +++ b/core/src/test/scala/integration/kafka/api/RackAwareAutoTopicCreationTest.scala @@ -44,7 +44,7 @@ class RackAwareAutoTopicCreationTest extends KafkaServerTestHarness with RackAwa @Test def testAutoCreateTopic() { - val producer = TestUtils.createNewProducer(brokerList, retries = 5) + val producer = TestUtils.createProducer(brokerList, retries = 5) try { // Send a message to auto-create the topic val record = new ProducerRecord(topic, null, "key".getBytes, "value".getBytes) diff --git a/core/src/test/scala/integration/kafka/api/SaslClientsWithInvalidCredentialsTest.scala b/core/src/test/scala/integration/kafka/api/SaslClientsWithInvalidCredentialsTest.scala index 79166270cb9cb..b79b679351ff3 100644 --- a/core/src/test/scala/integration/kafka/api/SaslClientsWithInvalidCredentialsTest.scala +++ b/core/src/test/scala/integration/kafka/api/SaslClientsWithInvalidCredentialsTest.scala @@ -25,7 +25,7 @@ import org.apache.kafka.common.errors.SaslAuthenticationException import org.apache.kafka.common.serialization.ByteArrayDeserializer import org.junit.{After, Before, Test} import org.junit.Assert._ -import kafka.admin.ConsumerGroupCommand.{ConsumerGroupCommandOptions, KafkaConsumerGroupService} +import kafka.admin.ConsumerGroupCommand.{ConsumerGroupCommandOptions, ConsumerGroupService} import kafka.server.KafkaConfig import kafka.utils.{JaasTestUtils, TestUtils} import kafka.zk.ConfigEntityChangeNotificationZNode @@ -169,7 +169,7 @@ class SaslClientsWithInvalidCredentialsTest extends IntegrationTestHarness with "--group", "test.group", "--command-config", propsFile.getAbsolutePath) val opts = new ConsumerGroupCommandOptions(cgcArgs) - val consumerGroupService = new KafkaConsumerGroupService(opts) + val consumerGroupService = new ConsumerGroupService(opts) val consumer = consumers.head consumer.subscribe(List(topic).asJava) @@ -227,7 +227,7 @@ class SaslClientsWithInvalidCredentialsTest extends IntegrationTestHarness with private def createTransactionalProducer(): KafkaProducer[Array[Byte], Array[Byte]] = { producerConfig.setProperty(ProducerConfig.TRANSACTIONAL_ID_CONFIG, "txclient-1") producerConfig.put(ProducerConfig.ENABLE_IDEMPOTENCE_CONFIG, "true") - val txProducer = TestUtils.createNewProducer(brokerList, + val txProducer = TestUtils.createProducer(brokerList, securityProtocol = this.securityProtocol, saslProperties = this.clientSaslProperties, retries = 1000, diff --git a/core/src/test/scala/integration/kafka/api/SaslEndToEndAuthorizationTest.scala b/core/src/test/scala/integration/kafka/api/SaslEndToEndAuthorizationTest.scala index 643cd4ce6afbb..c252f815fa1fc 100644 --- a/core/src/test/scala/integration/kafka/api/SaslEndToEndAuthorizationTest.scala +++ b/core/src/test/scala/integration/kafka/api/SaslEndToEndAuthorizationTest.scala @@ -65,7 +65,7 @@ abstract class SaslEndToEndAuthorizationTest extends EndToEndAuthorizationTest { consumer2Config.remove(SaslConfigs.SASL_JAAS_CONFIG) consumer2Config.remove(SaslConfigs.SASL_CLIENT_CALLBACK_HANDLER_CLASS) - val consumer2 = TestUtils.createNewConsumer(brokerList, + val consumer2 = TestUtils.createConsumer(brokerList, securityProtocol = securityProtocol, trustStoreFile = trustStoreFile, saslProperties = clientSaslProperties, diff --git a/core/src/test/scala/integration/kafka/api/SaslMultiMechanismConsumerTest.scala b/core/src/test/scala/integration/kafka/api/SaslMultiMechanismConsumerTest.scala index 6ae99df7e31ef..a7e33e0c5bfaf 100644 --- a/core/src/test/scala/integration/kafka/api/SaslMultiMechanismConsumerTest.scala +++ b/core/src/test/scala/integration/kafka/api/SaslMultiMechanismConsumerTest.scala @@ -50,12 +50,12 @@ class SaslMultiMechanismConsumerTest extends BaseConsumerTest with SaslSetup { val plainSaslConsumer = consumers.head val gssapiSaslProperties = kafkaClientSaslProperties("GSSAPI", dynamicJaasConfig = true) - val gssapiSaslProducer = TestUtils.createNewProducer(brokerList, + val gssapiSaslProducer = TestUtils.createProducer(brokerList, securityProtocol = this.securityProtocol, trustStoreFile = this.trustStoreFile, saslProperties = Some(gssapiSaslProperties)) producers += gssapiSaslProducer - val gssapiSaslConsumer = TestUtils.createNewConsumer(brokerList, + val gssapiSaslConsumer = TestUtils.createConsumer(brokerList, securityProtocol = this.securityProtocol, trustStoreFile = this.trustStoreFile, saslProperties = Some(gssapiSaslProperties)) diff --git a/core/src/test/scala/integration/kafka/api/SaslSslAdminClientIntegrationTest.scala b/core/src/test/scala/integration/kafka/api/SaslSslAdminClientIntegrationTest.scala index b3572c0665b62..3b63613419dfb 100644 --- a/core/src/test/scala/integration/kafka/api/SaslSslAdminClientIntegrationTest.scala +++ b/core/src/test/scala/integration/kafka/api/SaslSslAdminClientIntegrationTest.scala @@ -15,13 +15,15 @@ package kafka.api import java.io.File import java.util -import kafka.security.auth.{All, Allow, Alter, AlterConfigs, Authorizer, ClusterAction, Create, Delete, Deny, Describe, Group, Literal, Operation, PermissionType, SimpleAclAuthorizer, Topic, Prefixed, Acl => AuthAcl, Resource => AuthResource} +import kafka.security.auth.{All, Allow, Alter, AlterConfigs, Authorizer, ClusterAction, Create, Delete, Deny, Describe, Group, Operation, PermissionType, SimpleAclAuthorizer, Topic, Acl => AuthAcl, Resource => AuthResource} import kafka.server.KafkaConfig import kafka.utils.{CoreUtils, JaasTestUtils, TestUtils} +import kafka.utils.TestUtils._ + import org.apache.kafka.clients.admin.{AdminClient, CreateAclsOptions, DeleteAclsOptions} import org.apache.kafka.common.acl._ import org.apache.kafka.common.errors.{ClusterAuthorizationException, InvalidRequestException} -import org.apache.kafka.common.resource.{Resource, ResourceFilter, ResourceNameType, ResourceType} +import org.apache.kafka.common.resource.{PatternType, ResourcePattern, ResourcePatternFilter, ResourceType} import org.apache.kafka.common.security.auth.{KafkaPrincipal, SecurityProtocol} import org.junit.Assert.assertEquals import org.junit.{After, Assert, Before, Test} @@ -89,19 +91,19 @@ class SaslSslAdminClientIntegrationTest extends AdminClientIntegrationTest with closeSasl() } - val anyAcl = new AclBinding(new Resource(ResourceType.TOPIC, "*", ResourceNameType.LITERAL), + val anyAcl = new AclBinding(new ResourcePattern(ResourceType.TOPIC, "*", PatternType.LITERAL), new AccessControlEntry("User:*", "*", AclOperation.ALL, AclPermissionType.ALLOW)) - val acl2 = new AclBinding(new Resource(ResourceType.TOPIC, "mytopic2", ResourceNameType.LITERAL), + val acl2 = new AclBinding(new ResourcePattern(ResourceType.TOPIC, "mytopic2", PatternType.LITERAL), new AccessControlEntry("User:ANONYMOUS", "*", AclOperation.WRITE, AclPermissionType.ALLOW)) - val acl3 = new AclBinding(new Resource(ResourceType.TOPIC, "mytopic3", ResourceNameType.LITERAL), + val acl3 = new AclBinding(new ResourcePattern(ResourceType.TOPIC, "mytopic3", PatternType.LITERAL), new AccessControlEntry("User:ANONYMOUS", "*", AclOperation.READ, AclPermissionType.ALLOW)) - val fooAcl = new AclBinding(new Resource(ResourceType.TOPIC, "foobar", ResourceNameType.LITERAL), + val fooAcl = new AclBinding(new ResourcePattern(ResourceType.TOPIC, "foobar", PatternType.LITERAL), new AccessControlEntry("User:ANONYMOUS", "*", AclOperation.READ, AclPermissionType.ALLOW)) - val prefixAcl = new AclBinding(new Resource(ResourceType.TOPIC, "mytopic", ResourceNameType.PREFIXED), + val prefixAcl = new AclBinding(new ResourcePattern(ResourceType.TOPIC, "mytopic", PatternType.PREFIXED), new AccessControlEntry("User:ANONYMOUS", "*", AclOperation.READ, AclPermissionType.ALLOW)) - val transactionalIdAcl = new AclBinding(new Resource(ResourceType.TRANSACTIONAL_ID, "transactional_id", ResourceNameType.LITERAL), + val transactionalIdAcl = new AclBinding(new ResourcePattern(ResourceType.TRANSACTIONAL_ID, "transactional_id", PatternType.LITERAL), new AccessControlEntry("User:ANONYMOUS", "*", AclOperation.WRITE, AclPermissionType.ALLOW)) - val groupAcl = new AclBinding(new Resource(ResourceType.GROUP, "*", ResourceNameType.LITERAL), + val groupAcl = new AclBinding(new ResourcePattern(ResourceType.GROUP, "*", PatternType.LITERAL), new AccessControlEntry("User:*", "*", AclOperation.ALL, AclPermissionType.ALLOW)) @Test @@ -111,7 +113,7 @@ class SaslSslAdminClientIntegrationTest extends AdminClientIntegrationTest with val results = client.createAcls(List(acl2, acl3).asJava) assertEquals(Set(acl2, acl3), results.values.keySet().asScala) results.values.values().asScala.foreach(value => value.get) - val aclUnknown = new AclBinding(new Resource(ResourceType.TOPIC, "mytopic3", ResourceNameType.LITERAL), + val aclUnknown = new AclBinding(new ResourcePattern(ResourceType.TOPIC, "mytopic3", PatternType.LITERAL), new AccessControlEntry("User:ANONYMOUS", "*", AclOperation.UNKNOWN, AclPermissionType.ALLOW)) val results2 = client.createAcls(List(aclUnknown).asJava) assertEquals(Set(aclUnknown), results2.values.keySet().asScala) @@ -132,9 +134,9 @@ class SaslSslAdminClientIntegrationTest extends AdminClientIntegrationTest with waitForDescribeAcls(client, acl2.toFilter, Set(acl2)) waitForDescribeAcls(client, transactionalIdAcl.toFilter, Set(transactionalIdAcl)) - val filterA = new AclBindingFilter(new ResourceFilter(ResourceType.GROUP, null, ResourceNameType.LITERAL), AccessControlEntryFilter.ANY) - val filterB = new AclBindingFilter(new ResourceFilter(ResourceType.TOPIC, "mytopic2", ResourceNameType.LITERAL), AccessControlEntryFilter.ANY) - val filterC = new AclBindingFilter(new ResourceFilter(ResourceType.TRANSACTIONAL_ID, null, ResourceNameType.LITERAL), AccessControlEntryFilter.ANY) + val filterA = new AclBindingFilter(new ResourcePatternFilter(ResourceType.GROUP, null, PatternType.LITERAL), AccessControlEntryFilter.ANY) + val filterB = new AclBindingFilter(new ResourcePatternFilter(ResourceType.TOPIC, "mytopic2", PatternType.LITERAL), AccessControlEntryFilter.ANY) + val filterC = new AclBindingFilter(new ResourcePatternFilter(ResourceType.TRANSACTIONAL_ID, null, PatternType.LITERAL), AccessControlEntryFilter.ANY) waitForDescribeAcls(client, filterA, Set(groupAcl)) waitForDescribeAcls(client, filterC, Set(transactionalIdAcl)) @@ -154,13 +156,13 @@ class SaslSslAdminClientIntegrationTest extends AdminClientIntegrationTest with client = AdminClient.create(createConfig()) ensureAcls(Set(anyAcl, acl2, fooAcl, prefixAcl)) - val allTopicAcls = new AclBindingFilter(new ResourceFilter(ResourceType.TOPIC, null, ResourceNameType.ANY), AccessControlEntryFilter.ANY) - val allLiteralTopicAcls = new AclBindingFilter(new ResourceFilter(ResourceType.TOPIC, null, ResourceNameType.LITERAL), AccessControlEntryFilter.ANY) - val allPrefixedTopicAcls = new AclBindingFilter(new ResourceFilter(ResourceType.TOPIC, null, ResourceNameType.PREFIXED), AccessControlEntryFilter.ANY) - val literalMyTopic2Acls = new AclBindingFilter(new ResourceFilter(ResourceType.TOPIC, "mytopic2", ResourceNameType.LITERAL), AccessControlEntryFilter.ANY) - val prefixedMyTopicAcls = new AclBindingFilter(new ResourceFilter(ResourceType.TOPIC, "mytopic", ResourceNameType.PREFIXED), AccessControlEntryFilter.ANY) - val allMyTopic2Acls = new AclBindingFilter(new ResourceFilter(ResourceType.TOPIC, "mytopic2", ResourceNameType.ANY), AccessControlEntryFilter.ANY) - val allFooTopicAcls = new AclBindingFilter(new ResourceFilter(ResourceType.TOPIC, "foobar", ResourceNameType.ANY), AccessControlEntryFilter.ANY) + val allTopicAcls = new AclBindingFilter(new ResourcePatternFilter(ResourceType.TOPIC, null, PatternType.ANY), AccessControlEntryFilter.ANY) + val allLiteralTopicAcls = new AclBindingFilter(new ResourcePatternFilter(ResourceType.TOPIC, null, PatternType.LITERAL), AccessControlEntryFilter.ANY) + val allPrefixedTopicAcls = new AclBindingFilter(new ResourcePatternFilter(ResourceType.TOPIC, null, PatternType.PREFIXED), AccessControlEntryFilter.ANY) + val literalMyTopic2Acls = new AclBindingFilter(new ResourcePatternFilter(ResourceType.TOPIC, "mytopic2", PatternType.LITERAL), AccessControlEntryFilter.ANY) + val prefixedMyTopicAcls = new AclBindingFilter(new ResourcePatternFilter(ResourceType.TOPIC, "mytopic", PatternType.PREFIXED), AccessControlEntryFilter.ANY) + val allMyTopic2Acls = new AclBindingFilter(new ResourcePatternFilter(ResourceType.TOPIC, "mytopic2", PatternType.MATCH), AccessControlEntryFilter.ANY) + val allFooTopicAcls = new AclBindingFilter(new ResourcePatternFilter(ResourceType.TOPIC, "foobar", PatternType.MATCH), AccessControlEntryFilter.ANY) assertEquals(Set(anyAcl), getAcls(anyAcl.toFilter)) assertEquals(Set(prefixAcl), getAcls(prefixAcl.toFilter)) @@ -181,9 +183,9 @@ class SaslSslAdminClientIntegrationTest extends AdminClientIntegrationTest with client = AdminClient.create(createConfig()) ensureAcls(Set(anyAcl, acl2, fooAcl, prefixAcl)) - val allTopicAcls = new AclBindingFilter(new ResourceFilter(ResourceType.TOPIC, null, ResourceNameType.ANY), AccessControlEntryFilter.ANY) - val allLiteralTopicAcls = new AclBindingFilter(new ResourceFilter(ResourceType.TOPIC, null, ResourceNameType.LITERAL), AccessControlEntryFilter.ANY) - val allPrefixedTopicAcls = new AclBindingFilter(new ResourceFilter(ResourceType.TOPIC, null, ResourceNameType.PREFIXED), AccessControlEntryFilter.ANY) + val allTopicAcls = new AclBindingFilter(new ResourcePatternFilter(ResourceType.TOPIC, null, PatternType.MATCH), AccessControlEntryFilter.ANY) + val allLiteralTopicAcls = new AclBindingFilter(new ResourcePatternFilter(ResourceType.TOPIC, null, PatternType.LITERAL), AccessControlEntryFilter.ANY) + val allPrefixedTopicAcls = new AclBindingFilter(new ResourcePatternFilter(ResourceType.TOPIC, null, PatternType.PREFIXED), AccessControlEntryFilter.ANY) // Delete only ACLs on literal 'mytopic2' topic var deleted = client.deleteAcls(List(acl2.toFilter).asJava).all().get().asScala.toSet @@ -231,11 +233,11 @@ class SaslSslAdminClientIntegrationTest extends AdminClientIntegrationTest with client = AdminClient.create(createConfig()) ensureAcls(Set(anyAcl, acl2, fooAcl, prefixAcl)) // <-- prefixed exists, but should never be returned. - val allTopicAcls = new AclBindingFilter(new ResourceFilter(ResourceType.TOPIC, null, ResourceNameType.ANY), AccessControlEntryFilter.ANY) - val legacyAllTopicAcls = new AclBindingFilter(new ResourceFilter(ResourceType.TOPIC, null, ResourceNameType.LITERAL), AccessControlEntryFilter.ANY) - val legacyMyTopic2Acls = new AclBindingFilter(new ResourceFilter(ResourceType.TOPIC, "mytopic2", ResourceNameType.LITERAL), AccessControlEntryFilter.ANY) - val legacyAnyTopicAcls = new AclBindingFilter(new ResourceFilter(ResourceType.TOPIC, "*", ResourceNameType.LITERAL), AccessControlEntryFilter.ANY) - val legacyFooTopicAcls = new AclBindingFilter(new ResourceFilter(ResourceType.TOPIC, "foobar", ResourceNameType.LITERAL), AccessControlEntryFilter.ANY) + val allTopicAcls = new AclBindingFilter(new ResourcePatternFilter(ResourceType.TOPIC, null, PatternType.MATCH), AccessControlEntryFilter.ANY) + val legacyAllTopicAcls = new AclBindingFilter(new ResourcePatternFilter(ResourceType.TOPIC, null, PatternType.LITERAL), AccessControlEntryFilter.ANY) + val legacyMyTopic2Acls = new AclBindingFilter(new ResourcePatternFilter(ResourceType.TOPIC, "mytopic2", PatternType.LITERAL), AccessControlEntryFilter.ANY) + val legacyAnyTopicAcls = new AclBindingFilter(new ResourcePatternFilter(ResourceType.TOPIC, "*", PatternType.LITERAL), AccessControlEntryFilter.ANY) + val legacyFooTopicAcls = new AclBindingFilter(new ResourcePatternFilter(ResourceType.TOPIC, "foobar", PatternType.LITERAL), AccessControlEntryFilter.ANY) assertEquals(Set(anyAcl, acl2, fooAcl), getAcls(legacyAllTopicAcls)) assertEquals(Set(acl2), getAcls(legacyMyTopic2Acls)) @@ -266,9 +268,9 @@ class SaslSslAdminClientIntegrationTest extends AdminClientIntegrationTest with @Test def testAttemptToCreateInvalidAcls(): Unit = { client = AdminClient.create(createConfig()) - val clusterAcl = new AclBinding(new Resource(ResourceType.CLUSTER, "foobar", ResourceNameType.LITERAL), + val clusterAcl = new AclBinding(new ResourcePattern(ResourceType.CLUSTER, "foobar", PatternType.LITERAL), new AccessControlEntry("User:ANONYMOUS", "*", AclOperation.READ, AclPermissionType.ALLOW)) - val emptyResourceNameAcl = new AclBinding(new Resource(ResourceType.TOPIC, "", ResourceNameType.LITERAL), + val emptyResourceNameAcl = new AclBinding(new ResourcePattern(ResourceType.TOPIC, "", PatternType.LITERAL), new AccessControlEntry("User:ANONYMOUS", "*", AclOperation.READ, AclPermissionType.ALLOW)) val results = client.createAcls(List(clusterAcl, emptyResourceNameAcl).asJava, new CreateAclsOptions()) assertEquals(Set(clusterAcl, emptyResourceNameAcl), results.values.keySet().asScala) @@ -336,7 +338,7 @@ class SaslSslAdminClientIntegrationTest extends AdminClientIntegrationTest with private def testAclGet(expectAuth: Boolean): Unit = { TestUtils.waitUntilTrue(() => { - val userAcl = new AclBinding(new Resource(ResourceType.TOPIC, "*", ResourceNameType.LITERAL), + val userAcl = new AclBinding(new ResourcePattern(ResourceType.TOPIC, "*", PatternType.LITERAL), new AccessControlEntry("User:*", "*", AclOperation.ALL, AclPermissionType.ALLOW)) val results = client.describeAcls(userAcl.toFilter) if (expectAuth) { diff --git a/core/src/test/scala/integration/kafka/api/TransactionsBounceTest.scala b/core/src/test/scala/integration/kafka/api/TransactionsBounceTest.scala index 110f60953cc19..a65cc2ee6717f 100644 --- a/core/src/test/scala/integration/kafka/api/TransactionsBounceTest.scala +++ b/core/src/test/scala/integration/kafka/api/TransactionsBounceTest.scala @@ -160,7 +160,7 @@ class TransactionsBounceTest extends KafkaServerTestHarness { props.put(ConsumerConfig.HEARTBEAT_INTERVAL_MS_CONFIG, "3000") props.put(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG, "earliest") - val consumer = TestUtils.createNewConsumer(TestUtils.getBrokerListStrFromServers(servers), groupId = groupId, + val consumer = TestUtils.createConsumer(TestUtils.getBrokerListStrFromServers(servers), groupId = groupId, securityProtocol = SecurityProtocol.PLAINTEXT, props = Some(props)) consumer.subscribe(topics.asJava) consumer diff --git a/core/src/test/scala/integration/kafka/api/TransactionsTest.scala b/core/src/test/scala/integration/kafka/api/TransactionsTest.scala index 8435e5a3a6c15..461d3b61f0c5e 100644 --- a/core/src/test/scala/integration/kafka/api/TransactionsTest.scala +++ b/core/src/test/scala/integration/kafka/api/TransactionsTest.scala @@ -575,7 +575,7 @@ class TransactionsTest extends KafkaServerTestHarness { props.put(ConsumerConfig.ISOLATION_LEVEL_CONFIG, "read_committed") props.put(ConsumerConfig.ENABLE_AUTO_COMMIT_CONFIG, "false") props.put(ConsumerConfig.MAX_POLL_RECORDS_CONFIG, maxPollRecords.toString) - val consumer = TestUtils.createNewConsumer(TestUtils.getBrokerListStrFromServers(servers), + val consumer = TestUtils.createConsumer(TestUtils.getBrokerListStrFromServers(servers), groupId = group, securityProtocol = SecurityProtocol.PLAINTEXT, props = Some(props)) transactionalConsumers += consumer consumer @@ -585,7 +585,7 @@ class TransactionsTest extends KafkaServerTestHarness { val props = new Properties() props.put(ConsumerConfig.ISOLATION_LEVEL_CONFIG, "read_uncommitted") props.put(ConsumerConfig.ENABLE_AUTO_COMMIT_CONFIG, "false") - val consumer = TestUtils.createNewConsumer(TestUtils.getBrokerListStrFromServers(servers), + val consumer = TestUtils.createConsumer(TestUtils.getBrokerListStrFromServers(servers), groupId = group, securityProtocol = SecurityProtocol.PLAINTEXT, props = Some(props)) nonTransactionalConsumers += consumer consumer diff --git a/core/src/test/scala/integration/kafka/server/DynamicBrokerReconfigurationTest.scala b/core/src/test/scala/integration/kafka/server/DynamicBrokerReconfigurationTest.scala index 45b3fdc74bd61..45fbcbac56e36 100644 --- a/core/src/test/scala/integration/kafka/server/DynamicBrokerReconfigurationTest.scala +++ b/core/src/test/scala/integration/kafka/server/DynamicBrokerReconfigurationTest.scala @@ -126,14 +126,15 @@ class DynamicBrokerReconfigurationTest extends ZooKeeperTestHarness with SaslSet props ++= securityProps(sslProperties1, KEYSTORE_PROPS, listenerPrefix(SecureExternal)) val kafkaConfig = KafkaConfig.fromProps(props) + configureDynamicKeystoreInZooKeeper(kafkaConfig, sslProperties1) servers += TestUtils.createServer(kafkaConfig) } + TestUtils.createTopic(zkClient, topic, numPartitions, replicationFactor = numServers, servers) TestUtils.createTopic(zkClient, Topic.GROUP_METADATA_TOPIC_NAME, OffsetConfig.DefaultOffsetsTopicNumPartitions, replicationFactor = numServers, servers, servers.head.groupCoordinator.offsetsTopicConfigs) - TestUtils.createTopic(zkClient, topic, numPartitions, replicationFactor = numServers, servers) createAdminClient(SecurityProtocol.SSL, SecureInternal) TestMetricsReporter.testReporters.clear() @@ -369,6 +370,7 @@ class DynamicBrokerReconfigurationTest extends ZooKeeperTestHarness with SaslSet props.put(KafkaConfig.LogPreAllocateProp, true.toString) props.put(KafkaConfig.LogMessageTimestampTypeProp, TimestampType.LOG_APPEND_TIME.toString) props.put(KafkaConfig.LogMessageTimestampDifferenceMaxMsProp, "1000") + props.put(KafkaConfig.LogMessageDownConversionEnableProp, "false") reconfigureServers(props, perBrokerConfig = false, (KafkaConfig.LogSegmentBytesProp, "4000")) // Verify that all broker defaults have been updated @@ -731,17 +733,25 @@ class DynamicBrokerReconfigurationTest extends ZooKeeperTestHarness with SaslSet // Verify that producer connections fail since advertised listener is invalid val bootstrap = TestUtils.bootstrapServers(servers, new ListenerName(SecureExternal)) .replaceAll(invalidHost, "localhost") // allow bootstrap connection to succeed - val producer1 = ProducerBuilder().trustStoreProps(sslProperties1).maxRetries(0).bootstrapServers(bootstrap).build() + val producer1 = ProducerBuilder().trustStoreProps(sslProperties1) + .maxRetries(0) + .requestTimeoutMs(1000) + .bootstrapServers(bootstrap) + .build() - val sendFuture = verifyConnectionFailure(producer1) + assertTrue(intercept[ExecutionException] { + producer1.send(new ProducerRecord(topic, "key", "value")).get(2, TimeUnit.SECONDS) + }.getCause.isInstanceOf[org.apache.kafka.common.errors.TimeoutException]) alterAdvertisedListener(adminClient, externalAdminClient, invalidHost, "localhost") servers.foreach(validateEndpointsInZooKeeper(_, endpoints => !endpoints.contains(invalidHost))) // Verify that produce/consume work now + val topic2 = "testtopic2" + TestUtils.createTopic(zkClient, topic2, numPartitions, replicationFactor = numServers, servers) val producer = ProducerBuilder().trustStoreProps(sslProperties1).maxRetries(0).build() - val consumer = ConsumerBuilder("group2").trustStoreProps(sslProperties1).topic(topic).build() - verifyProduceConsume(producer, consumer, 10, topic) + val consumer = ConsumerBuilder("group2").trustStoreProps(sslProperties1).topic(topic2).build() + verifyProduceConsume(producer, consumer, 10, topic2) // Verify updating inter-broker listener val props = new Properties @@ -754,9 +764,6 @@ class DynamicBrokerReconfigurationTest extends ZooKeeperTestHarness with SaslSet assertTrue(s"Unexpected exception ${e.getCause}", e.getCause.isInstanceOf[InvalidRequestException]) servers.foreach(server => assertEquals(SecureInternal, server.config.interBrokerListenerName.value)) } - - // Verify that the other send did not complete - verifyTimeout(sendFuture) } @Test @@ -777,21 +784,12 @@ class DynamicBrokerReconfigurationTest extends ZooKeeperTestHarness with SaslSet val props = adminZkClient.fetchEntityConfig(ConfigType.Broker, server.config.brokerId.toString) val propsEncodedWithOldSecret = props.clone().asInstanceOf[Properties] val config = server.config - val secret = config.passwordEncoderSecret.getOrElse(throw new IllegalStateException("Password encoder secret not configured")) val oldSecret = "old-dynamic-config-secret" config.dynamicConfig.staticBrokerConfigs.put(KafkaConfig.PasswordEncoderOldSecretProp, oldSecret) val passwordConfigs = props.asScala.filterKeys(DynamicBrokerConfig.isPasswordConfig) assertTrue("Password configs not found", passwordConfigs.nonEmpty) - val passwordDecoder = new PasswordEncoder(secret, - config.passwordEncoderKeyFactoryAlgorithm, - config.passwordEncoderCipherAlgorithm, - config.passwordEncoderKeyLength, - config.passwordEncoderIterations) - val passwordEncoder = new PasswordEncoder(new Password(oldSecret), - config.passwordEncoderKeyFactoryAlgorithm, - config.passwordEncoderCipherAlgorithm, - config.passwordEncoderKeyLength, - config.passwordEncoderIterations) + val passwordDecoder = createPasswordEncoder(config, config.passwordEncoderSecret) + val passwordEncoder = createPasswordEncoder(config, Some(new Password(oldSecret))) passwordConfigs.foreach { case (name, value) => val decoded = passwordDecoder.decode(value).value propsEncodedWithOldSecret.put(name, passwordEncoder.encode(new Password(decoded))) @@ -824,10 +822,23 @@ class DynamicBrokerReconfigurationTest extends ZooKeeperTestHarness with SaslSet verifyAddListener("SASL_PLAINTEXT", SecurityProtocol.SASL_PLAINTEXT, Seq("GSSAPI")) //verifyRemoveListener("SASL_SSL", SecurityProtocol.SASL_SSL, Seq("SCRAM-SHA-512", "SCRAM-SHA-256", "PLAIN")) verifyRemoveListener("SASL_PLAINTEXT", SecurityProtocol.SASL_PLAINTEXT, Seq("GSSAPI")) + + // Verify that a listener added to a subset of servers doesn't cause any issues + // when metadata is processed by the client. + addListener(servers.tail, "SCRAM_LISTENER", SecurityProtocol.SASL_PLAINTEXT, Seq("SCRAM-SHA-256")) + val bootstrap = TestUtils.bootstrapServers(servers.tail, new ListenerName("SCRAM_LISTENER")) + val producer = ProducerBuilder().bootstrapServers(bootstrap) + .securityProtocol(SecurityProtocol.SASL_PLAINTEXT) + .saslMechanism("SCRAM-SHA-256") + .maxRetries(1000) + .build() + val partitions = producer.partitionsFor(topic).asScala + assertEquals(0, partitions.count(p => p.leader != null && p.leader.id == servers.head.config.brokerId)) + assertTrue("Did not find partitions with no leader", partitions.exists(_.leader == null)) } - private def verifyAddListener(listenerName: String, securityProtocol: SecurityProtocol, - saslMechanisms: Seq[String]): Unit = { + private def addListener(servers: Seq[KafkaServer], listenerName: String, securityProtocol: SecurityProtocol, + saslMechanisms: Seq[String]): Unit = { val config = servers.head.config val existingListenerCount = config.listeners.size val listeners = config.listeners @@ -867,14 +878,7 @@ class DynamicBrokerReconfigurationTest extends ZooKeeperTestHarness with SaslSet } }), "Listener not created") - if (saslMechanisms.nonEmpty) - saslMechanisms.foreach { mechanism => - verifyListener(securityProtocol, Some(mechanism), s"add-listener-group-$securityProtocol-$mechanism") - } - else - verifyListener(securityProtocol, None, s"add-listener-group-$securityProtocol") - - val brokerConfigs = describeConfig(adminClients.head).entries.asScala + val brokerConfigs = describeConfig(adminClients.head, servers).entries.asScala props.asScala.foreach { case (name, value) => val entry = brokerConfigs.find(_.name == name).getOrElse(throw new IllegalArgumentException(s"Config not found $name")) if (DynamicBrokerConfig.isPasswordConfig(name) || name == unknownConfig) @@ -884,6 +888,17 @@ class DynamicBrokerReconfigurationTest extends ZooKeeperTestHarness with SaslSet } } + private def verifyAddListener(listenerName: String, securityProtocol: SecurityProtocol, + saslMechanisms: Seq[String]): Unit = { + addListener(servers, listenerName, securityProtocol, saslMechanisms) + if (saslMechanisms.nonEmpty) + saslMechanisms.foreach { mechanism => + verifyListener(securityProtocol, Some(mechanism), s"add-listener-group-$securityProtocol-$mechanism") + } + else + verifyListener(securityProtocol, None, s"add-listener-group-$securityProtocol") + } + private def verifyRemoveListener(listenerName: String, securityProtocol: SecurityProtocol, saslMechanisms: Seq[String]): Unit = { val saslMechanism = if (saslMechanisms.isEmpty) "" else saslMechanisms.head @@ -928,13 +943,15 @@ class DynamicBrokerReconfigurationTest extends ZooKeeperTestHarness with SaslSet val consumerFuture = verifyConnectionFailure(consumer1) // Test that other listeners still work + val topic2 = "testtopic2" + TestUtils.createTopic(zkClient, topic2, numPartitions, replicationFactor = numServers, servers) val producer2 = ProducerBuilder().trustStoreProps(sslProperties1).maxRetries(0).build() val consumer2 = ConsumerBuilder(s"remove-listener-group2-$securityProtocol") .trustStoreProps(sslProperties1) - .topic(topic) + .topic(topic2) .autoOffsetReset("latest") .build() - verifyProduceConsume(producer2, consumer2, numRecords = 10, topic) + verifyProduceConsume(producer2, consumer2, numRecords = 10, topic2) // Verify that producer/consumer using old listener don't work verifyTimeout(producerFuture) @@ -1013,7 +1030,7 @@ class DynamicBrokerReconfigurationTest extends ZooKeeperTestHarness with SaslSet } } - private def describeConfig(adminClient: AdminClient): Config = { + private def describeConfig(adminClient: AdminClient, servers: Seq[KafkaServer] = this.servers): Config = { val configResources = servers.map { server => new ConfigResource(ConfigResource.Type.BROKER, server.config.brokerId.toString) } @@ -1160,12 +1177,39 @@ class DynamicBrokerReconfigurationTest extends ZooKeeperTestHarness with SaslSet private def listenerPrefix(name: String): String = new ListenerName(name).configPrefix - private def configureDynamicKeystoreInZooKeeper(kafkaConfig: KafkaConfig, brokers: Seq[Int], sslProperties: Properties): Unit = { + private def configureDynamicKeystoreInZooKeeper(kafkaConfig: KafkaConfig, sslProperties: Properties): Unit = { + val externalListenerPrefix = listenerPrefix(SecureExternal) val sslStoreProps = new Properties - sslStoreProps ++= securityProps(sslProperties, KEYSTORE_PROPS, listenerPrefix(SecureExternal)) - val persistentProps = kafkaConfig.dynamicConfig.toPersistentProps(sslStoreProps, perBrokerConfig = true) + sslStoreProps ++= securityProps(sslProperties, KEYSTORE_PROPS, externalListenerPrefix) + sslStoreProps.put(KafkaConfig.PasswordEncoderSecretProp, kafkaConfig.passwordEncoderSecret.map(_.value).orNull) zkClient.makeSurePersistentPathExists(ConfigEntityChangeNotificationZNode.path) - adminZkClient.changeBrokerConfig(brokers, persistentProps) + + val args = Array("--zookeeper", kafkaConfig.zkConnect, + "--alter", "--add-config", sslStoreProps.asScala.map { case (k, v) => s"$k=$v" }.mkString(","), + "--entity-type", "brokers", + "--entity-name", kafkaConfig.brokerId.toString) + ConfigCommand.main(args) + + val passwordEncoder = createPasswordEncoder(kafkaConfig, kafkaConfig.passwordEncoderSecret) + val brokerProps = adminZkClient.fetchEntityConfig("brokers", kafkaConfig.brokerId.toString) + assertEquals(4, brokerProps.size) + assertEquals(sslProperties.get(SSL_KEYSTORE_TYPE_CONFIG), + brokerProps.getProperty(s"$externalListenerPrefix$SSL_KEYSTORE_TYPE_CONFIG")) + assertEquals(sslProperties.get(SSL_KEYSTORE_LOCATION_CONFIG), + brokerProps.getProperty(s"$externalListenerPrefix$SSL_KEYSTORE_LOCATION_CONFIG")) + assertEquals(sslProperties.get(SSL_KEYSTORE_PASSWORD_CONFIG), + passwordEncoder.decode(brokerProps.getProperty(s"$externalListenerPrefix$SSL_KEYSTORE_PASSWORD_CONFIG"))) + assertEquals(sslProperties.get(SSL_KEY_PASSWORD_CONFIG), + passwordEncoder.decode(brokerProps.getProperty(s"$externalListenerPrefix$SSL_KEY_PASSWORD_CONFIG"))) + } + + private def createPasswordEncoder(config: KafkaConfig, secret: Option[Password]): PasswordEncoder = { + val encoderSecret = secret.getOrElse(throw new IllegalStateException("Password encoder secret not configured")) + new PasswordEncoder(encoderSecret, + config.passwordEncoderKeyFactoryAlgorithm, + config.passwordEncoderCipherAlgorithm, + config.passwordEncoderKeyLength, + config.passwordEncoderIterations) } private def waitForConfig(propName: String, propValue: String, maxWaitMs: Long = 10000): Unit = { @@ -1324,13 +1368,16 @@ class DynamicBrokerReconfigurationTest extends ZooKeeperTestHarness with SaslSet private case class ProducerBuilder() extends ClientBuilder[KafkaProducer[String, String]] { private var _retries = 0 private var _acks = -1 + private var _requestTimeoutMs = 30000L def maxRetries(retries: Int): ProducerBuilder = { _retries = retries; this } def acks(acks: Int): ProducerBuilder = { _acks = acks; this } + def requestTimeoutMs(timeoutMs: Long): ProducerBuilder = { _requestTimeoutMs = timeoutMs; this } override def build(): KafkaProducer[String, String] = { - val producer = TestUtils.createNewProducer(bootstrapServers, + val producer = TestUtils.createProducer(bootstrapServers, acks = _acks, + requestTimeoutMs = _requestTimeoutMs, retries = _retries, securityProtocol = _securityProtocol, trustStoreFile = Some(trustStoreFile1), @@ -1354,7 +1401,7 @@ class DynamicBrokerReconfigurationTest extends ZooKeeperTestHarness with SaslSet override def build(): KafkaConsumer[String, String] = { val consumerProps = propsOverride consumerProps.put(ConsumerConfig.ENABLE_AUTO_COMMIT_CONFIG, _enableAutoCommit.toString) - val consumer = TestUtils.createNewConsumer(bootstrapServers, + val consumer = TestUtils.createConsumer(bootstrapServers, group, autoOffsetReset = _autoOffsetReset, securityProtocol = _securityProtocol, diff --git a/core/src/test/scala/integration/kafka/server/GssapiAuthenticationTest.scala b/core/src/test/scala/integration/kafka/server/GssapiAuthenticationTest.scala new file mode 100644 index 0000000000000..a0f70943d91c3 --- /dev/null +++ b/core/src/test/scala/integration/kafka/server/GssapiAuthenticationTest.scala @@ -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 kafka.server + +import java.net.InetSocketAddress +import java.util.Properties +import java.util.concurrent.{Executors, TimeUnit} + +import kafka.api.{Both, IntegrationTestHarness, SaslSetup} +import kafka.utils.TestUtils +import org.apache.kafka.clients.CommonClientConfigs +import org.apache.kafka.common.config.SaslConfigs +import org.apache.kafka.common.network._ +import org.apache.kafka.common.security.{JaasContext, TestSecurityConfig} +import org.apache.kafka.common.security.auth.SecurityProtocol +import org.junit.Assert._ +import org.junit.{After, Before, Test} + +class GssapiAuthenticationTest extends IntegrationTestHarness with SaslSetup { + + override val producerCount = 0 + override val consumerCount = 0 + override val serverCount = 1 + override protected def securityProtocol = SecurityProtocol.SASL_PLAINTEXT + + private val kafkaClientSaslMechanism = "GSSAPI" + private val kafkaServerSaslMechanisms = List("GSSAPI") + + private val numThreads = 10 + private val executor = Executors.newFixedThreadPool(numThreads) + private val clientConfig: Properties = new Properties + private var serverAddr: InetSocketAddress = _ + + @Before + override def setUp() { + startSasl(jaasSections(kafkaServerSaslMechanisms, Option(kafkaClientSaslMechanism), Both)) + super.setUp() + serverAddr = new InetSocketAddress("localhost", + servers.head.boundPort(ListenerName.forSecurityProtocol(SecurityProtocol.SASL_PLAINTEXT))) + + clientConfig.put(CommonClientConfigs.SECURITY_PROTOCOL_CONFIG, SecurityProtocol.SASL_PLAINTEXT.name) + clientConfig.put(SaslConfigs.SASL_MECHANISM, kafkaClientSaslMechanism) + clientConfig.put(SaslConfigs.SASL_JAAS_CONFIG, jaasClientLoginModule(kafkaClientSaslMechanism)) + clientConfig.put(CommonClientConfigs.CONNECTIONS_MAX_IDLE_MS_CONFIG, "5000") + } + + @After + override def tearDown(): Unit = { + executor.shutdownNow() + super.tearDown() + closeSasl() + } + + /** + * Tests that Kerberos replay error `Request is a replay (34)` is not handled as an authentication exception + * since replay detection used to detect DoS attacks may occasionally reject valid concurrent requests. + */ + @Test + def testRequestIsAReplay(): Unit = { + val successfulAuthsPerThread = 10 + val futures = (0 until numThreads).map(_ => executor.submit(new Runnable { + override def run(): Unit = verifyRetriableFailuresDuringAuthentication(successfulAuthsPerThread) + })) + futures.foreach(_.get(60, TimeUnit.SECONDS)) + assertEquals(0, TestUtils.totalMetricValue(servers.head, "failed-authentication-total")) + val successfulAuths = TestUtils.totalMetricValue(servers.head, "successful-authentication-total") + assertTrue("Too few authentications: " + successfulAuths, successfulAuths > successfulAuthsPerThread * numThreads) + } + + /** + * Tests that Kerberos error `Server not found in Kerberos database (7)` is handled + * as a fatal authentication failure. + */ + @Test + def testServerNotFoundInKerberosDatabase(): Unit = { + val jaasConfig = clientConfig.getProperty(SaslConfigs.SASL_JAAS_CONFIG) + val invalidServiceConfig = jaasConfig.replace("serviceName=\"kafka\"", "serviceName=\"invalid-service\"") + clientConfig.put(SaslConfigs.SASL_JAAS_CONFIG, invalidServiceConfig) + clientConfig.put(SaslConfigs.SASL_KERBEROS_SERVICE_NAME, "invalid-service") + verifyNonRetriableAuthenticationFailure() + } + + /** + * Verifies that any exceptions during authentication with the current `clientConfig` are + * notified with disconnect state `AUTHENTICATE` (and not `AUTHENTICATION_FAILED`). This + * is to ensure that NetworkClient doesn't handle this as a fatal authentication failure, + * but as a transient I/O exception. So Producer/Consumer/AdminClient will retry + * any operation based on their configuration until timeout and will not propagate + * the exception to the application. + */ + private def verifyRetriableFailuresDuringAuthentication(numSuccessfulAuths: Int): Unit = { + val selector = createSelector() + try { + var actualSuccessfulAuths = 0 + while (actualSuccessfulAuths < numSuccessfulAuths) { + val nodeId = actualSuccessfulAuths.toString + selector.connect(nodeId, serverAddr, 1024, 1024) + TestUtils.waitUntilTrue(() => { + selector.poll(100) + val disconnectState = selector.disconnected().get(nodeId) + // Verify that disconnect state is not AUTHENTICATION_FAILED + if (disconnectState != null) + assertEquals(s"Authentication failed with exception ${disconnectState.exception()}", + ChannelState.State.AUTHENTICATE, disconnectState.state()) + selector.isChannelReady(nodeId) || disconnectState != null + }, "Client not ready or disconnected within timeout") + if (selector.isChannelReady(nodeId)) + actualSuccessfulAuths += 1 + selector.close(nodeId) + } + } finally { + selector.close() + } + } + + /** + * Verifies that authentication with the current `clientConfig` results in disconnection and that + * the disconnection is notified with disconnect state `AUTHENTICATION_FAILED`. This is to ensure + * that NetworkClient handles this as a fatal authentication failure that is propagated to + * applications by Producer/Consumer/AdminClient without retrying and waiting for timeout. + */ + private def verifyNonRetriableAuthenticationFailure(): Unit = { + val selector = createSelector() + val nodeId = "1" + selector.connect(nodeId, serverAddr, 1024, 1024) + TestUtils.waitUntilTrue(() => { + selector.poll(100) + val disconnectState = selector.disconnected().get(nodeId) + if (disconnectState != null) + assertEquals(ChannelState.State.AUTHENTICATION_FAILED, disconnectState.state()) + disconnectState != null + }, "Client not disconnected within timeout") + } + + private def createSelector(): Selector = { + val channelBuilder = ChannelBuilders.clientChannelBuilder(securityProtocol, + JaasContext.Type.CLIENT, new TestSecurityConfig(clientConfig), null, kafkaClientSaslMechanism, true) + NetworkTestUtils.createSelector(channelBuilder) + } +} diff --git a/core/src/test/scala/integration/kafka/server/MultipleListenersWithSameSecurityProtocolBaseTest.scala b/core/src/test/scala/integration/kafka/server/MultipleListenersWithSameSecurityProtocolBaseTest.scala index 50ebaa40f9312..851ac5178e64a 100644 --- a/core/src/test/scala/integration/kafka/server/MultipleListenersWithSameSecurityProtocolBaseTest.scala +++ b/core/src/test/scala/integration/kafka/server/MultipleListenersWithSameSecurityProtocolBaseTest.scala @@ -130,10 +130,10 @@ abstract class MultipleListenersWithSameSecurityProtocolBaseTest extends ZooKeep TestUtils.createTopic(zkClient, topic, 2, 2, servers) val clientMetadata = ClientMetadata(listenerName, mechanism, topic) - producers(clientMetadata) = TestUtils.createNewProducer(bootstrapServers, acks = -1, + producers(clientMetadata) = TestUtils.createProducer(bootstrapServers, acks = -1, securityProtocol = endPoint.securityProtocol, trustStoreFile = trustStoreFile, saslProperties = saslProps) - consumers(clientMetadata) = TestUtils.createNewConsumer(bootstrapServers, groupId = clientMetadata.toString, + consumers(clientMetadata) = TestUtils.createConsumer(bootstrapServers, groupId = clientMetadata.toString, securityProtocol = endPoint.securityProtocol, trustStoreFile = trustStoreFile, saslProperties = saslProps) } diff --git a/core/src/test/scala/integration/kafka/server/ReplicaFetcherThreadFatalErrorTest.scala b/core/src/test/scala/integration/kafka/server/ReplicaFetcherThreadFatalErrorTest.scala deleted file mode 100644 index 5e81dc5a739c6..0000000000000 --- a/core/src/test/scala/integration/kafka/server/ReplicaFetcherThreadFatalErrorTest.scala +++ /dev/null @@ -1,145 +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.util.concurrent.atomic.AtomicBoolean - -import kafka.cluster.BrokerEndPoint -import kafka.server.ReplicaFetcherThread.{FetchRequest, PartitionData} -import kafka.utils.{Exit, TestUtils} -import kafka.utils.TestUtils.createBrokerConfigs -import kafka.zk.ZooKeeperTestHarness -import org.apache.kafka.common.TopicPartition -import org.apache.kafka.common.internals.FatalExitError -import org.apache.kafka.common.metrics.Metrics -import org.apache.kafka.common.protocol.Errors -import org.apache.kafka.common.requests.FetchResponse -import org.apache.kafka.common.utils.Time -import org.junit.{After, Test} - -import scala.collection.Map -import scala.collection.JavaConverters._ -import scala.concurrent.Future - -class ReplicaFetcherThreadFatalErrorTest extends ZooKeeperTestHarness { - - private var brokers: Seq[KafkaServer] = null - @volatile private var shutdownCompleted = false - - @After - override def tearDown() { - Exit.resetExitProcedure() - TestUtils.shutdownServers(brokers) - super.tearDown() - } - - /** - * Verifies that a follower shuts down if the offset for an `added partition` is out of range and if a fatal - * exception is thrown from `handleOffsetOutOfRange`. It's a bit tricky to ensure that there are no deadlocks - * when the shutdown hook is invoked and hence this test. - */ - @Test - def testFatalErrorInAddPartitions(): Unit = { - - // Unlike `TestUtils.createTopic`, this doesn't wait for metadata propagation as the broker shuts down before - // the metadata is propagated. - def createTopic(topic: String): Unit = { - adminZkClient.createTopic(topic, partitions = 1, replicationFactor = 2) - TestUtils.waitUntilLeaderIsElectedOrChanged(zkClient, topic, 0) - } - - val props = createBrokerConfigs(2, zkConnect) - brokers = props.map(KafkaConfig.fromProps).map(config => createServer(config, { params => - import params._ - new ReplicaFetcherThread(threadName, fetcherId, sourceBroker, config, replicaManager, metrics, time, quotaManager) { - override def handleOffsetOutOfRange(topicPartition: TopicPartition): Long = throw new FatalExitError - override def addPartitions(partitionAndOffsets: Map[TopicPartition, Long]): Unit = - super.addPartitions(partitionAndOffsets.mapValues(_ => -1)) - } - })) - createTopic("topic") - TestUtils.waitUntilTrue(() => shutdownCompleted, "Shutdown of follower did not complete") - } - - /** - * Verifies that a follower shuts down if the offset of a partition in the fetch response is out of range and if a - * fatal exception is thrown from `handleOffsetOutOfRange`. It's a bit tricky to ensure that there are no deadlocks - * when the shutdown hook is invoked and hence this test. - */ - @Test - def testFatalErrorInProcessFetchRequest(): Unit = { - val props = createBrokerConfigs(2, zkConnect) - brokers = props.map(KafkaConfig.fromProps).map(config => createServer(config, { params => - import params._ - new ReplicaFetcherThread(threadName, fetcherId, sourceBroker, config, replicaManager, metrics, time, quotaManager) { - override def handleOffsetOutOfRange(topicPartition: TopicPartition): Long = throw new FatalExitError - override protected def fetch(fetchRequest: FetchRequest): Seq[(TopicPartition, PartitionData)] = { - fetchRequest.underlying.fetchData.asScala.keys.toSeq.map { tp => - (tp, new PartitionData(new FetchResponse.PartitionData(Errors.OFFSET_OUT_OF_RANGE, - FetchResponse.INVALID_HIGHWATERMARK, FetchResponse.INVALID_LAST_STABLE_OFFSET, FetchResponse.INVALID_LOG_START_OFFSET, null, null))) - } - } - } - })) - TestUtils.createTopic(zkClient, "topic", numPartitions = 1, replicationFactor = 2, servers = brokers) - TestUtils.waitUntilTrue(() => shutdownCompleted, "Shutdown of follower did not complete") - } - - private case class FetcherThreadParams(threadName: String, fetcherId: Int, sourceBroker: BrokerEndPoint, - replicaManager: ReplicaManager, metrics: Metrics, time: Time, - quotaManager: ReplicationQuotaManager) - - private def createServer(config: KafkaConfig, fetcherThread: FetcherThreadParams => ReplicaFetcherThread): KafkaServer = { - val time = Time.SYSTEM - val server = new KafkaServer(config, time) { - - override def createReplicaManager(isShuttingDown: AtomicBoolean): ReplicaManager = { - new ReplicaManager(config, metrics, time, zkClient, kafkaScheduler, logManager, isShuttingDown, - quotaManagers, new BrokerTopicStats, metadataCache, logDirFailureChannel) { - - override protected def createReplicaFetcherManager(metrics: Metrics, time: Time, threadNamePrefix: Option[String], - quotaManager: ReplicationQuotaManager) = - new ReplicaFetcherManager(config, this, metrics, time, threadNamePrefix, quotaManager) { - override def createFetcherThread(fetcherId: Int, sourceBroker: BrokerEndPoint): AbstractFetcherThread = { - val prefix = threadNamePrefix.map(tp => s"$tp:").getOrElse("") - val threadName = s"${prefix}ReplicaFetcherThread-$fetcherId-${sourceBroker.id}" - fetcherThread(FetcherThreadParams(threadName, fetcherId, sourceBroker, replicaManager, metrics, - time, quotaManager)) - } - } - } - } - - } - - Exit.setExitProcedure { (_, _) => - import scala.concurrent.ExecutionContext.Implicits._ - // Run in a separate thread like shutdown hooks - Future { - server.shutdown() - shutdownCompleted = true - } - // Sleep until interrupted to emulate the fact that `System.exit()` never returns - Thread.sleep(Long.MaxValue) - throw new AssertionError - } - server.startup() - server - } - -} diff --git a/core/src/test/scala/integration/kafka/server/ScramServerStartupTest.scala b/core/src/test/scala/integration/kafka/server/ScramServerStartupTest.scala index 18b4f8e23b84d..85028744ca370 100644 --- a/core/src/test/scala/integration/kafka/server/ScramServerStartupTest.scala +++ b/core/src/test/scala/integration/kafka/server/ScramServerStartupTest.scala @@ -58,16 +58,9 @@ class ScramServerStartupTest extends IntegrationTestHarness with SaslSetup { @Test def testAuthentications(): Unit = { - val successfulAuths = totalAuthentications("successful-authentication-total") + val successfulAuths = TestUtils.totalMetricValue(servers.head, "successful-authentication-total") assertTrue("No successful authentications", successfulAuths > 0) - val failedAuths = totalAuthentications("failed-authentication-total") + val failedAuths = TestUtils.totalMetricValue(servers.head, "failed-authentication-total") assertEquals(0, failedAuths) } - - private def totalAuthentications(metricName: String): Int = { - val allMetrics = servers.head.metrics.metrics - val totalAuthCount = allMetrics.values().asScala.filter(_.metricName().name() == metricName) - .foldLeft(0.0)((total, metric) => total + metric.metricValue.asInstanceOf[Double]) - totalAuthCount.toInt - } } diff --git a/core/src/test/scala/integration/kafka/tools/MirrorMakerIntegrationTest.scala b/core/src/test/scala/integration/kafka/tools/MirrorMakerIntegrationTest.scala index 1f9851dadef56..0a178195bef2a 100644 --- a/core/src/test/scala/integration/kafka/tools/MirrorMakerIntegrationTest.scala +++ b/core/src/test/scala/integration/kafka/tools/MirrorMakerIntegrationTest.scala @@ -18,13 +18,13 @@ package kafka.tools import java.util.Properties -import kafka.consumer.ConsumerTimeoutException import kafka.integration.KafkaServerTestHarness import kafka.server.KafkaConfig -import kafka.tools.MirrorMaker.{MirrorMakerNewConsumer, MirrorMakerProducer} +import kafka.tools.MirrorMaker.{ConsumerWrapper, MirrorMakerProducer} import kafka.utils.TestUtils import org.apache.kafka.clients.consumer.{ConsumerConfig, KafkaConsumer} import org.apache.kafka.clients.producer.{ProducerConfig, ProducerRecord} +import org.apache.kafka.common.errors.TimeoutException import org.apache.kafka.common.serialization.{ByteArrayDeserializer, ByteArraySerializer} import org.junit.Test @@ -54,7 +54,7 @@ class MirrorMakerIntegrationTest extends KafkaServerTestHarness { consumerProps.put(ConsumerConfig.BOOTSTRAP_SERVERS_CONFIG, brokerList) val consumer = new KafkaConsumer(consumerProps, new ByteArrayDeserializer, new ByteArrayDeserializer) - val mirrorMakerConsumer = new MirrorMakerNewConsumer(consumer, None, whitelistOpt = Some("another_topic,new.*,foo")) + val mirrorMakerConsumer = new ConsumerWrapper(consumer, None, whitelistOpt = Some("another_topic,new.*,foo")) mirrorMakerConsumer.init() try { TestUtils.waitUntilTrue(() => { @@ -63,7 +63,7 @@ class MirrorMakerIntegrationTest extends KafkaServerTestHarness { data.topic == topic && new String(data.value) == msg } catch { // this exception is thrown if no record is returned within a short timeout, so safe to ignore - case _: ConsumerTimeoutException => false + case _: TimeoutException => false } }, "MirrorMaker consumer should read the expected message from the expected topic within the timeout") } finally consumer.close() diff --git a/core/src/test/scala/kafka/common/InterBrokerSendThreadTest.scala b/core/src/test/scala/kafka/common/InterBrokerSendThreadTest.scala index 710686693e32e..6838653ad137e 100644 --- a/core/src/test/scala/kafka/common/InterBrokerSendThreadTest.scala +++ b/core/src/test/scala/kafka/common/InterBrokerSendThreadTest.scala @@ -32,11 +32,12 @@ class InterBrokerSendThreadTest { private val time = new MockTime() private val networkClient = EasyMock.createMock(classOf[NetworkClient]) private val completionHandler = new StubCompletionHandler + private val requestTimeoutMs = 1000 @Test def shouldNotSendAnythingWhenNoRequests(): Unit = { val sendThread = new InterBrokerSendThread("name", networkClient, time) { - override val unsentExpiryMs: Int = 1000 + override val requestTimeoutMs: Int = InterBrokerSendThreadTest.this.requestTimeoutMs override def generateRequests() = mutable.Iterable.empty } @@ -58,16 +59,18 @@ class InterBrokerSendThreadTest { val node = new Node(1, "", 8080) val handler = RequestAndCompletionHandler(node, request, completionHandler) val sendThread = new InterBrokerSendThread("name", networkClient, time) { - override val unsentExpiryMs: Int = 1000 + override val requestTimeoutMs: Int = InterBrokerSendThreadTest.this.requestTimeoutMs override def generateRequests() = List[RequestAndCompletionHandler](handler) } - val clientRequest = new ClientRequest("dest", request, 0, "1", 0, true, handler.handler) + val clientRequest = new ClientRequest("dest", request, 0, "1", 0, true, + requestTimeoutMs, handler.handler) EasyMock.expect(networkClient.newClientRequest(EasyMock.eq("1"), EasyMock.same(handler.request), EasyMock.anyLong(), EasyMock.eq(true), + EasyMock.eq(requestTimeoutMs), EasyMock.same(handler.handler))) .andReturn(clientRequest) @@ -93,16 +96,18 @@ class InterBrokerSendThreadTest { val node = new Node(1, "", 8080) val requestAndCompletionHandler = RequestAndCompletionHandler(node, request, completionHandler) val sendThread = new InterBrokerSendThread("name", networkClient, time) { - override val unsentExpiryMs: Int = 1000 + override val requestTimeoutMs: Int = InterBrokerSendThreadTest.this.requestTimeoutMs override def generateRequests() = List[RequestAndCompletionHandler](requestAndCompletionHandler) } - val clientRequest = new ClientRequest("dest", request, 0, "1", 0, true, requestAndCompletionHandler.handler) + val clientRequest = new ClientRequest("dest", request, 0, "1", 0, true, + requestTimeoutMs, requestAndCompletionHandler.handler) EasyMock.expect(networkClient.newClientRequest(EasyMock.eq("1"), EasyMock.same(requestAndCompletionHandler.request), EasyMock.anyLong(), EasyMock.eq(true), + EasyMock.eq(requestTimeoutMs), EasyMock.same(requestAndCompletionHandler.handler))) .andReturn(clientRequest) @@ -135,17 +140,19 @@ class InterBrokerSendThreadTest { val node = new Node(1, "", 8080) val handler = RequestAndCompletionHandler(node, request, completionHandler) val sendThread = new InterBrokerSendThread("name", networkClient, time) { - override val unsentExpiryMs: Int = 1000 + override val requestTimeoutMs: Int = InterBrokerSendThreadTest.this.requestTimeoutMs override def generateRequests() = List[RequestAndCompletionHandler](handler) } - val clientRequest = new ClientRequest("dest", request, 0, "1", time.milliseconds(), true, handler.handler) + val clientRequest = new ClientRequest("dest", request, 0, "1", time.milliseconds(), true, + requestTimeoutMs, handler.handler) time.sleep(1500) EasyMock.expect(networkClient.newClientRequest(EasyMock.eq("1"), EasyMock.same(handler.request), EasyMock.eq(time.milliseconds()), EasyMock.eq(true), + EasyMock.eq(requestTimeoutMs), EasyMock.same(handler.handler))) .andReturn(clientRequest) diff --git a/core/src/test/scala/kafka/security/auth/ResourceTest.scala b/core/src/test/scala/kafka/security/auth/ResourceTest.scala new file mode 100644 index 0000000000000..1b7a8330e7c3d --- /dev/null +++ b/core/src/test/scala/kafka/security/auth/ResourceTest.scala @@ -0,0 +1,67 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package kafka.security.auth + +import kafka.common.KafkaException +import org.apache.kafka.common.resource.PatternType.{LITERAL, PREFIXED} +import org.junit.Test +import org.junit.Assert._ + +class ResourceTest { + @Test(expected = classOf[KafkaException]) + def shouldThrowOnTwoPartStringWithUnknownResourceType(): Unit = { + Resource.fromString("Unknown:fred") + } + + @Test(expected = classOf[KafkaException]) + def shouldThrowOnBadResourceTypeSeparator(): Unit = { + Resource.fromString("Topic-fred") + } + + @Test + def shouldParseOldTwoPartString(): Unit = { + assertEquals(Resource(Group, "fred", LITERAL), Resource.fromString("Group:fred")) + assertEquals(Resource(Topic, "t", LITERAL), Resource.fromString("Topic:t")) + } + + @Test + def shouldParseOldTwoPartWithEmbeddedSeparators(): Unit = { + assertEquals(Resource(Group, ":This:is:a:weird:group:name:", LITERAL), Resource.fromString("Group::This:is:a:weird:group:name:")) + } + + @Test + def shouldParseThreePartString(): Unit = { + assertEquals(Resource(Group, "fred", PREFIXED), Resource.fromString("Group:PREFIXED:fred")) + assertEquals(Resource(Topic, "t", LITERAL), Resource.fromString("Topic:LITERAL:t")) + } + + @Test + def shouldParseThreePartWithEmbeddedSeparators(): Unit = { + assertEquals(Resource(Group, ":This:is:a:weird:group:name:", PREFIXED), Resource.fromString("Group:PREFIXED::This:is:a:weird:group:name:")) + assertEquals(Resource(Group, ":This:is:a:weird:group:name:", LITERAL), Resource.fromString("Group:LITERAL::This:is:a:weird:group:name:")) + } + + @Test + def shouldRoundTripViaString(): Unit = { + val expected = Resource(Group, "fred", PREFIXED) + + val actual = Resource.fromString(expected.toString) + + assertEquals(expected, actual) + } +} \ No newline at end of file diff --git a/core/src/test/scala/kafka/tools/ReplicaVerificationToolTest.scala b/core/src/test/scala/kafka/tools/ReplicaVerificationToolTest.scala index 211413a341fe4..f69c9092265bd 100644 --- a/core/src/test/scala/kafka/tools/ReplicaVerificationToolTest.scala +++ b/core/src/test/scala/kafka/tools/ReplicaVerificationToolTest.scala @@ -17,11 +17,10 @@ package kafka.tools -import kafka.api.FetchResponsePartitionData -import kafka.common.TopicAndPartition -import kafka.message.ByteBufferMessageSet +import org.apache.kafka.common.TopicPartition import org.apache.kafka.common.protocol.Errors -import org.apache.kafka.common.record.{CompressionType, SimpleRecord, MemoryRecords} +import org.apache.kafka.common.record.{CompressionType, MemoryRecords, SimpleRecord} +import org.apache.kafka.common.requests.FetchResponse import org.junit.Test import org.junit.Assert.assertTrue @@ -32,12 +31,12 @@ class ReplicaVerificationToolTest { val sb = new StringBuilder val expectedReplicasPerTopicAndPartition = Map( - TopicAndPartition("a", 0) -> 3, - TopicAndPartition("a", 1) -> 3, - TopicAndPartition("b", 0) -> 2 + new TopicPartition("a", 0) -> 3, + new TopicPartition("a", 1) -> 3, + new TopicPartition("b", 0) -> 2 ) - val replicaBuffer = new ReplicaBuffer(expectedReplicasPerTopicAndPartition, Map.empty, 2, Map.empty, 0, 0) + val replicaBuffer = new ReplicaBuffer(expectedReplicasPerTopicAndPartition, Map.empty, 2, 0) expectedReplicasPerTopicAndPartition.foreach { case (tp, numReplicas) => (0 until numReplicas).foreach { replicaId => val records = (0 to 5).map { index => @@ -45,8 +44,9 @@ class ReplicaVerificationToolTest { } val initialOffset = 4 val memoryRecords = MemoryRecords.withRecords(initialOffset, CompressionType.NONE, records: _*) - replicaBuffer.addFetchedData(tp, replicaId, new FetchResponsePartitionData(Errors.NONE, hw = 20, - new ByteBufferMessageSet(memoryRecords.buffer))) + val partitionData = new FetchResponse.PartitionData(Errors.NONE, 20, 20, 0L, null, memoryRecords) + + replicaBuffer.addFetchedData(tp, replicaId, partitionData) } } @@ -55,7 +55,7 @@ class ReplicaVerificationToolTest { // If you change this assertion, you should verify that the replica_verification_test.py system test still passes assertTrue(s"Max lag information should be in output: `$output`", - output.endsWith(": max lag is 10 for partition a-0 at offset 10 among 3 partitions")) + output.endsWith(": max lag is 10 for partition a-1 at offset 10 among 3 partitions")) } } diff --git a/core/src/test/scala/kafka/tools/TestLogCleaning.scala b/core/src/test/scala/kafka/tools/TestLogCleaning.scala deleted file mode 100755 index 4ad6629eae1d8..0000000000000 --- a/core/src/test/scala/kafka/tools/TestLogCleaning.scala +++ /dev/null @@ -1,323 +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 joptsimple.OptionParser -import java.util.Properties -import java.util.Random -import java.io._ - -import kafka.consumer._ -import kafka.serializer._ -import kafka.utils._ -import kafka.log.Log -import org.apache.kafka.clients.producer.{KafkaProducer, ProducerConfig, ProducerRecord} -import org.apache.kafka.common.record.FileRecords - -import scala.collection.JavaConverters._ - -/** - * This is a torture test that runs against an existing broker. Here is how it works: - * - * It produces a series of specially formatted messages to one or more partitions. Each message it produces - * it logs out to a text file. The messages have a limited set of keys, so there is duplication in the key space. - * - * The broker will clean its log as the test runs. - * - * When the specified number of messages have been produced we create a consumer and consume all the messages in the topic - * and write that out to another text file. - * - * Using a stable unix sort we sort both the producer log of what was sent and the consumer log of what was retrieved by the message key. - * Then we compare the final message in both logs for each key. If this final message is not the same for all keys we - * print an error and exit with exit code 1, otherwise we print the size reduction and exit with exit code 0. - */ -object TestLogCleaning { - - def main(args: Array[String]) { - val parser = new OptionParser(false) - val numMessagesOpt = parser.accepts("messages", "The number of messages to send or consume.") - .withRequiredArg - .describedAs("count") - .ofType(classOf[java.lang.Long]) - .defaultsTo(Long.MaxValue) - val 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") - .ofType(classOf[java.lang.Integer]) - .defaultsTo(5) - val brokerOpt = parser.accepts("broker", "Url to connect to.") - .withRequiredArg - .describedAs("url") - .ofType(classOf[String]) - val topicsOpt = parser.accepts("topics", "The number of topics to test.") - .withRequiredArg - .describedAs("count") - .ofType(classOf[java.lang.Integer]) - .defaultsTo(1) - val percentDeletesOpt = parser.accepts("percent-deletes", "The percentage of updates that are deletes.") - .withRequiredArg - .describedAs("percent") - .ofType(classOf[java.lang.Integer]) - .defaultsTo(0) - val zkConnectOpt = parser.accepts("zk", "Zk url.") - .withRequiredArg - .describedAs("url") - .ofType(classOf[String]) - val sleepSecsOpt = parser.accepts("sleep", "Time to sleep between production and consumption.") - .withRequiredArg - .describedAs("ms") - .ofType(classOf[java.lang.Integer]) - .defaultsTo(0) - val dumpOpt = parser.accepts("dump", "Dump the message contents of a topic partition that contains test data from this test to standard out.") - .withRequiredArg - .describedAs("directory") - .ofType(classOf[String]) - - val options = parser.parse(args:_*) - - if(args.length == 0) - CommandLineUtils.printUsageAndDie(parser, "An integration test for log cleaning.") - - if(options.has(dumpOpt)) { - dumpLog(new File(options.valueOf(dumpOpt))) - Exit.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, compressionType, dups, percentDeletes) - println("Sleeping for %d seconds...".format(sleepSecs)) - Thread.sleep(sleepSecs * 1000) - println("Consuming messages...") - val consumedDataFile = consumeMessages(zkUrl, topics) - - val producedLines = lineCount(producedDataFile) - val consumedLines = lineCount(consumedDataFile) - val reduction = 1.0 - consumedLines.toDouble/producedLines.toDouble - println("%d rows of data produced, %d rows of data consumed (%.1f%% reduction).".format(producedLines, consumedLines, 100 * reduction)) - - println("De-duplicating and validating output files...") - validateOutput(producedDataFile, consumedDataFile) - producedDataFile.delete() - consumedDataFile.delete() - } - - def dumpLog(dir: File) { - require(dir.exists, "Non-existent directory: " + dir.getAbsolutePath) - for (file <- dir.list.sorted; if file.endsWith(Log.LogFileSuffix)) { - val fileRecords = FileRecords.open(new File(dir, file)) - for (entry <- fileRecords.records.asScala) { - val key = TestUtils.readString(entry.key) - val content = - if (!entry.hasValue) - null - else - TestUtils.readString(entry.value) - println("offset = %s, key = %s, content = %s".format(entry.offset, key, content)) - } - } - } - - def lineCount(file: File): Int = io.Source.fromFile(file).getLines.size - - def validateOutput(producedDataFile: File, consumedDataFile: File) { - val producedReader = externalSort(producedDataFile) - val consumedReader = externalSort(consumedDataFile) - val produced = valuesIterator(producedReader) - val consumed = valuesIterator(consumedReader) - val producedDedupedFile = new File(producedDataFile.getAbsolutePath + ".deduped") - val producedDeduped = new BufferedWriter(new FileWriter(producedDedupedFile), 1024*1024) - val consumedDedupedFile = new File(consumedDataFile.getAbsolutePath + ".deduped") - val consumedDeduped = new BufferedWriter(new FileWriter(consumedDedupedFile), 1024*1024) - var total = 0 - var mismatched = 0 - while(produced.hasNext && consumed.hasNext) { - val p = produced.next() - producedDeduped.write(p.toString) - producedDeduped.newLine() - val c = consumed.next() - consumedDeduped.write(c.toString) - consumedDeduped.newLine() - if(p != c) - mismatched += 1 - total += 1 - } - producedDeduped.close() - consumedDeduped.close() - 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.") - require(mismatched == 0, "Non-zero number of row mismatches.") - // if all the checks worked out we can delete the deduped files - producedDedupedFile.delete() - consumedDedupedFile.delete() - } - - def valuesIterator(reader: BufferedReader) = { - new IteratorTemplate[TestRecord] { - def makeNext(): TestRecord = { - var next = readNext(reader) - while(next != null && next.delete) - next = readNext(reader) - if(next == null) - allDone() - else - next - } - } - } - - def readNext(reader: BufferedReader): TestRecord = { - var line = reader.readLine() - if(line == null) - return null - var curr = new TestRecord(line) - while(true) { - line = peekLine(reader) - if(line == null) - return curr - val next = new TestRecord(line) - if(next == null || next.topicAndKey != curr.topicAndKey) - return curr - curr = next - reader.readLine() - } - null - } - - def peekLine(reader: BufferedReader) = { - reader.mark(4096) - val line = reader.readLine - reader.reset() - line - } - - def externalSort(file: File): BufferedReader = { - val builder = new ProcessBuilder("sort", "--key=1,2", "--stable", "--buffer-size=20%", "--temporary-directory=" + System.getProperty("java.io.tmpdir"), file.getAbsolutePath) - val process = builder.start() - new Thread() { - override def run() { - val exitCode = process.waitFor() - if(exitCode != 0) { - System.err.println("Process exited abnormally.") - while(process.getErrorStream.available > 0) { - System.err.write(process.getErrorStream().read()) - } - } - } - }.start() - new BufferedReader(new InputStreamReader(process.getInputStream()), 10*1024*1024) - } - - def produceMessages(brokerUrl: String, - topics: Array[String], - messages: Long, - compressionType: String, - dups: Int, - percentDeletes: Int): File = { - val producerProps = new Properties - producerProps.setProperty(ProducerConfig.MAX_BLOCK_MS_CONFIG, Long.MaxValue.toString) - producerProps.setProperty(ProducerConfig.BOOTSTRAP_SERVERS_CONFIG, brokerUrl) - producerProps.put(ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG, "org.apache.kafka.common.serialization.ByteArraySerializer") - producerProps.put(ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG, "org.apache.kafka.common.serialization.ByteArraySerializer") - producerProps.setProperty(ProducerConfig.COMPRESSION_TYPE_CONFIG, compressionType) - val producer = new KafkaProducer[Array[Byte],Array[Byte]](producerProps) - val rand = new Random(1) - val keyCount = (messages / dups).toInt - val producedFile = File.createTempFile("kafka-log-cleaner-produced-", ".txt") - println("Logging produce requests to " + producedFile.getAbsolutePath) - val producedWriter = new BufferedWriter(new FileWriter(producedFile), 1024*1024) - for(i <- 0L until (messages * topics.length)) { - val topic = topics((i % topics.length).toInt) - val key = rand.nextInt(keyCount) - val delete = i % 100 < percentDeletes - val msg = - if(delete) - new ProducerRecord[Array[Byte],Array[Byte]](topic, key.toString.getBytes(), null) - else - 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() - } - producedWriter.close() - producer.close() - producedFile - } - - @deprecated("This method has been deprecated and will be removed in a future release.", "0.11.0.0") - def makeConsumer(zkUrl: String, topics: Array[String]): ZookeeperConsumerConnector = { - val consumerProps = new Properties - consumerProps.setProperty("group.id", "log-cleaner-test-" + new Random().nextInt(Int.MaxValue)) - consumerProps.setProperty("zookeeper.connect", zkUrl) - consumerProps.setProperty("consumer.timeout.ms", (20*1000).toString) - consumerProps.setProperty("auto.offset.reset", "smallest") - new ZookeeperConsumerConnector(new ConsumerConfig(consumerProps)) - } - - def consumeMessages(zkUrl: String, topics: Array[String]): File = { - val connector = makeConsumer(zkUrl, topics) - val streams = connector.createMessageStreams(topics.map(topic => (topic, 1)).toMap, new StringDecoder, new StringDecoder) - val consumedFile = File.createTempFile("kafka-log-cleaner-consumed-", ".txt") - println("Logging consumed messages to " + consumedFile.getAbsolutePath) - val consumedWriter = new BufferedWriter(new FileWriter(consumedFile)) - for(topic <- topics) { - val stream = streams(topic).head - try { - for(item <- stream) { - val delete = item.message == null - val value = if(delete) -1L else item.message.toLong - consumedWriter.write(TestRecord(topic, item.key.toInt, value, delete).toString) - consumedWriter.newLine() - } - } catch { - case _: ConsumerTimeoutException => - } - } - consumedWriter.close() - connector.shutdown() - consumedFile - } - -} - -case class TestRecord(topic: String, key: Int, value: Long, delete: Boolean) { - def this(pieces: Array[String]) = this(pieces(0), pieces(1).toInt, pieces(2).toLong, pieces(3) == "d") - def this(line: String) = this(line.split("\t")) - override def toString = topic + "\t" + key + "\t" + value + "\t" + (if(delete) "d" else "u") - def topicAndKey = topic + key -} diff --git a/core/src/test/scala/kafka/zk/ExtendedAclStoreTest.scala b/core/src/test/scala/kafka/zk/ExtendedAclStoreTest.scala new file mode 100644 index 0000000000000..ee6c399a6eace --- /dev/null +++ b/core/src/test/scala/kafka/zk/ExtendedAclStoreTest.scala @@ -0,0 +1,67 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package kafka.zk + +import kafka.security.auth.{Resource, Topic} +import org.apache.kafka.common.resource.PatternType.{LITERAL, PREFIXED} +import org.junit.Assert.assertEquals +import org.junit.Test + +class ExtendedAclStoreTest { + private val literalResource = Resource(Topic, "some-topic", LITERAL) + private val prefixedResource = Resource(Topic, "some-topic", PREFIXED) + private val store = new ExtendedAclStore(PREFIXED) + + @Test + def shouldHaveCorrectPaths(): Unit = { + assertEquals("/kafka-acl-extended/prefixed", store.aclPath) + assertEquals("/kafka-acl-extended/prefixed/Topic", store.path(Topic)) + assertEquals("/kafka-acl-extended-changes", store.changeStore.aclChangePath) + } + + @Test + def shouldHaveCorrectPatternType(): Unit = { + assertEquals(PREFIXED, store.patternType) + } + + @Test(expected = classOf[IllegalArgumentException]) + def shouldThrowIfConstructedWithLiteral(): Unit = { + new ExtendedAclStore(LITERAL) + } + + @Test(expected = classOf[IllegalArgumentException]) + def shouldThrowFromEncodeOnLiteral(): Unit = { + store.changeStore.createChangeNode(literalResource) + } + + @Test + def shouldWriteChangesToTheWritePath(): Unit = { + val changeNode = store.changeStore.createChangeNode(prefixedResource) + + assertEquals("/kafka-acl-extended-changes/acl_changes_", changeNode.path) + } + + @Test + def shouldRoundTripChangeNode(): Unit = { + val changeNode = store.changeStore.createChangeNode(prefixedResource) + + val actual = store.changeStore.decode(changeNode.bytes) + + assertEquals(prefixedResource, actual) + } +} \ No newline at end of file diff --git a/core/src/test/scala/kafka/zk/LiteralAclStoreTest.scala b/core/src/test/scala/kafka/zk/LiteralAclStoreTest.scala new file mode 100644 index 0000000000000..ec5de1be692ed --- /dev/null +++ b/core/src/test/scala/kafka/zk/LiteralAclStoreTest.scala @@ -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 kafka.zk + +import java.nio.charset.StandardCharsets.UTF_8 + +import kafka.security.auth.{Group, Resource, Topic} +import org.apache.kafka.common.resource.PatternType.{LITERAL, PREFIXED} +import org.junit.Assert.assertEquals +import org.junit.Test + +class LiteralAclStoreTest { + private val literalResource = Resource(Topic, "some-topic", LITERAL) + private val prefixedResource = Resource(Topic, "some-topic", PREFIXED) + private val store = LiteralAclStore + + @Test + def shouldHaveCorrectPaths(): Unit = { + assertEquals("/kafka-acl", store.aclPath) + assertEquals("/kafka-acl/Topic", store.path(Topic)) + assertEquals("/kafka-acl-changes", store.changeStore.aclChangePath) + } + + @Test + def shouldHaveCorrectPatternType(): Unit = { + assertEquals(LITERAL, store.patternType) + } + + @Test(expected = classOf[IllegalArgumentException]) + def shouldThrowFromEncodeOnNoneLiteral(): Unit = { + store.changeStore.createChangeNode(prefixedResource) + } + + @Test + def shouldWriteChangesToTheWritePath(): Unit = { + val changeNode = store.changeStore.createChangeNode(literalResource) + + assertEquals("/kafka-acl-changes/acl_changes_", changeNode.path) + } + + @Test + def shouldRoundTripChangeNode(): Unit = { + val changeNode = store.changeStore.createChangeNode(literalResource) + + val actual = store.changeStore.decode(changeNode.bytes) + + assertEquals(literalResource, actual) + } + + @Test + def shouldDecodeResourceUsingTwoPartLogic(): Unit = { + val resource = Resource(Group, "PREFIXED:this, including the PREFIXED part, is a valid two part group name", LITERAL) + val encoded = (resource.resourceType + Resource.Separator + resource.name).getBytes(UTF_8) + + val actual = store.changeStore.decode(encoded) + + assertEquals(resource, actual) + } +} \ No newline at end of file diff --git a/core/src/test/scala/other/kafka/ReplicationQuotasTestRig.scala b/core/src/test/scala/other/kafka/ReplicationQuotasTestRig.scala index 6fdad30216235..ee069e27935e9 100644 --- a/core/src/test/scala/other/kafka/ReplicationQuotasTestRig.scala +++ b/core/src/test/scala/other/kafka/ReplicationQuotasTestRig.scala @@ -126,7 +126,7 @@ object ReplicationQuotasTestRig { createTopic(zkClient, topicName, replicas, servers) println("Writing Data") - val producer = TestUtils.createNewProducer(TestUtils.getBrokerListStrFromServers(servers), retries = 5, acks = 0) + val producer = TestUtils.createProducer(TestUtils.getBrokerListStrFromServers(servers), retries = 5, acks = 0) (0 until config.msgsPerPartition).foreach { x => (0 until config.partitions).foreach { partition => producer.send(new ProducerRecord(topicName, partition, null, new Array[Byte](config.msgSize))) diff --git a/core/src/test/scala/other/kafka/TestCrcPerformance.scala b/core/src/test/scala/other/kafka/TestCrcPerformance.scala deleted file mode 100755 index daeecbd303044..0000000000000 --- a/core/src/test/scala/other/kafka/TestCrcPerformance.scala +++ /dev/null @@ -1,44 +0,0 @@ -/** - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package kafka.log - -import java.util.Random -import kafka.message._ -import org.apache.kafka.common.utils.Utils - -object TestCrcPerformance { - - def main(args: Array[String]): Unit = { - if(args.length < 2) - Utils.croak("USAGE: java " + getClass().getName() + " num_messages message_size") - val numMessages = args(0).toInt - val messageSize = args(1).toInt - - val content = new Array[Byte](messageSize) - new Random(1).nextBytes(content) - - // create message test - val start = System.nanoTime - for (_ <- 0 until numMessages) - new Message(content) - - val elapsed = System.nanoTime - start - println("%d messages created in %.2f seconds + (%.2f ns per message).".format(numMessages, elapsed / (1000.0*1000.0*1000.0), - elapsed / numMessages.toDouble)) - - } -} diff --git a/core/src/test/scala/other/kafka/TestLinearWriteSpeed.scala b/core/src/test/scala/other/kafka/TestLinearWriteSpeed.scala index 16325ee4ec53f..b385a2a4dd085 100755 --- a/core/src/test/scala/other/kafka/TestLinearWriteSpeed.scala +++ b/core/src/test/scala/other/kafka/TestLinearWriteSpeed.scala @@ -102,7 +102,7 @@ object TestLinearWriteSpeed { val compressionCodec = CompressionCodec.getCompressionCodec(options.valueOf(compressionCodecOpt)) val rand = new Random rand.nextBytes(buffer.array) - val numMessages = bufferSize / (messageSize + MessageSet.LogOverhead) + val numMessages = bufferSize / (messageSize + Records.LOG_OVERHEAD) val createTime = System.currentTimeMillis val messageSet = { val compressionType = CompressionType.forId(compressionCodec.codec) diff --git a/core/src/test/scala/other/kafka/TestOffsetManager.scala b/core/src/test/scala/other/kafka/TestOffsetManager.scala deleted file mode 100644 index c8f9397353e4d..0000000000000 --- a/core/src/test/scala/other/kafka/TestOffsetManager.scala +++ /dev/null @@ -1,309 +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 other.kafka - -import kafka.api._ -import kafka.utils.{Exit, ShutdownableThread, ZkUtils} -import org.apache.kafka.common.protocol.Errors - -import scala.collection._ -import kafka.client.ClientUtils -import joptsimple.OptionParser -import kafka.common.{OffsetAndMetadata, TopicAndPartition} -import kafka.network.BlockingChannel - -import scala.util.Random -import java.io.IOException - -import kafka.metrics.{KafkaMetricsGroup, KafkaTimer} -import java.util.concurrent.TimeUnit -import java.util.concurrent.atomic.AtomicInteger -import java.nio.channels.ClosedByInterruptException - - -object TestOffsetManager { - - val random = new Random - val SocketTimeoutMs = 10000 - - class StatsThread(reportingIntervalMs: Long, commitThreads: Seq[CommitThread], fetchThread: FetchThread) - extends ShutdownableThread("stats-thread") { - - def printStats() { - println("--------------------------------------------------------------------------------") - println("Aggregate stats for commits:") - println("Error count: %d; Max:%f; Min: %f; Mean: %f; Commit count: %d".format( - commitThreads.map(_.numErrors.get).sum, - commitThreads.map(_.timer.max()).max, - commitThreads.map(_.timer.min()).min, - commitThreads.map(_.timer.mean()).sum / commitThreads.size, - commitThreads.map(_.numCommits.get).sum)) - println("--------------------------------------------------------------------------------") - commitThreads.foreach(t => println(t.stats)) - println(fetchThread.stats) - } - - override def doWork() { - printStats() - Thread.sleep(reportingIntervalMs) - } - - } - - class CommitThread(id: Int, partitionCount: Int, commitIntervalMs: Long, zkUtils: ZkUtils) - extends ShutdownableThread("commit-thread") - with KafkaMetricsGroup { - - private val groupId = "group-" + id - private val metadata = "Metadata from commit thread " + id - private var offsetsChannel = ClientUtils.channelToOffsetManager(groupId, zkUtils, SocketTimeoutMs) - private var offset = 0L - val numErrors = new AtomicInteger(0) - val numCommits = new AtomicInteger(0) - val timer = newTimer("commit-thread", TimeUnit.MILLISECONDS, TimeUnit.SECONDS) - private val commitTimer = new KafkaTimer(timer) - val shutdownLock = new Object - - private def ensureConnected() { - if (!offsetsChannel.isConnected) - offsetsChannel = ClientUtils.channelToOffsetManager(groupId, zkUtils, SocketTimeoutMs) - } - - override def doWork() { - val commitRequest = OffsetCommitRequest(groupId, immutable.Map((1 to partitionCount).map(TopicAndPartition("topic-" + id, _) -> OffsetAndMetadata(offset, metadata)):_*)) - try { - ensureConnected() - offsetsChannel.send(commitRequest) - numCommits.getAndIncrement - commitTimer.time { - val response = OffsetCommitResponse.readFrom(offsetsChannel.receive().payload()) - if (response.commitStatus.exists(_._2 != Errors.NONE)) numErrors.getAndIncrement - } - offset += 1 - } - catch { - case _: ClosedByInterruptException => - offsetsChannel.disconnect() - case e2: IOException => - println("Commit thread %d: Error while committing offsets to %s:%d for group %s due to %s.".format(id, offsetsChannel.host, offsetsChannel.port, groupId, e2)) - offsetsChannel.disconnect() - } - finally { - Thread.sleep(commitIntervalMs) - } - } - - override def shutdown() { - super.shutdown() - awaitShutdown() - offsetsChannel.disconnect() - println("Commit thread %d ended. Last committed offset: %d.".format(id, offset)) - } - - def stats = { - "Commit thread %d :: Error count: %d; Max:%f; Min: %f; Mean: %f; Commit count: %d" - .format(id, numErrors.get(), timer.max(), timer.min(), timer.mean(), numCommits.get()) - } - } - - class FetchThread(numGroups: Int, fetchIntervalMs: Long, zkUtils: ZkUtils) - extends ShutdownableThread("fetch-thread") - with KafkaMetricsGroup { - - private val timer = newTimer("fetch-thread", TimeUnit.MILLISECONDS, TimeUnit.SECONDS) - private val fetchTimer = new KafkaTimer(timer) - - private val channels = mutable.Map[Int, BlockingChannel]() - private var metadataChannel = ClientUtils.channelToAnyBroker(zkUtils, SocketTimeoutMs) - - private val numErrors = new AtomicInteger(0) - - override def doWork() { - val id = random.nextInt().abs % numGroups - val group = "group-" + id - try { - metadataChannel.send(GroupCoordinatorRequest(group)) - val coordinatorId = GroupCoordinatorResponse.readFrom(metadataChannel.receive().payload()).coordinatorOpt.map(_.id).getOrElse(-1) - - val channel = if (channels.contains(coordinatorId)) - channels(coordinatorId) - else { - val newChannel = ClientUtils.channelToOffsetManager(group, zkUtils, SocketTimeoutMs) - channels.put(coordinatorId, newChannel) - newChannel - } - - try { - // send the offset fetch request - val fetchRequest = OffsetFetchRequest(group, Seq(TopicAndPartition("topic-"+id, 1))) - channel.send(fetchRequest) - - fetchTimer.time { - val response = OffsetFetchResponse.readFrom(channel.receive().payload()) - if (response.requestInfo.exists(_._2.error != Errors.NONE)) { - numErrors.getAndIncrement - } - } - } - catch { - case _: ClosedByInterruptException => - channel.disconnect() - channels.remove(coordinatorId) - case e2: IOException => - println("Error while fetching offset from %s:%d due to %s.".format(channel.host, channel.port, e2)) - channel.disconnect() - channels.remove(coordinatorId) - } - } - catch { - case _: IOException => - println("Error while querying %s:%d - shutting down query channel.".format(metadataChannel.host, metadataChannel.port)) - metadataChannel.disconnect() - println("Creating new query channel.") - metadataChannel = ClientUtils.channelToAnyBroker(zkUtils, SocketTimeoutMs) - } - finally { - Thread.sleep(fetchIntervalMs) - } - - } - - override def shutdown() { - super.shutdown() - awaitShutdown() - channels.foreach(_._2.disconnect()) - metadataChannel.disconnect() - } - - def stats = { - "Fetch thread :: Error count: %d; Max:%f; Min: %f; Mean: %f; Fetch count: %d" - .format(numErrors.get(), timer.max(), timer.min(), timer.mean(), timer.count()) - } - } - - def main(args: Array[String]) { - val parser = new OptionParser(false) - val zookeeperOpt = parser.accepts("zookeeper", "The ZooKeeper connection URL.") - .withRequiredArg - .describedAs("ZooKeeper URL") - .ofType(classOf[java.lang.String]) - .defaultsTo("localhost:2181") - - val commitIntervalOpt = parser.accepts("commit-interval-ms", "Offset commit interval.") - .withRequiredArg - .describedAs("interval") - .ofType(classOf[java.lang.Integer]) - .defaultsTo(100) - - val fetchIntervalOpt = parser.accepts("fetch-interval-ms", "Offset fetch interval.") - .withRequiredArg - .describedAs("interval") - .ofType(classOf[java.lang.Integer]) - .defaultsTo(1000) - - val numPartitionsOpt = parser.accepts("partition-count", "Number of partitions per commit.") - .withRequiredArg - .describedAs("interval") - .ofType(classOf[java.lang.Integer]) - .defaultsTo(1) - - val numThreadsOpt = parser.accepts("thread-count", "Number of commit threads.") - .withRequiredArg - .describedAs("threads") - .ofType(classOf[java.lang.Integer]) - .defaultsTo(1) - - val reportingIntervalOpt = parser.accepts("reporting-interval-ms", "Interval at which stats are reported.") - .withRequiredArg - .describedAs("interval (ms)") - .ofType(classOf[java.lang.Integer]) - .defaultsTo(3000) - - val helpOpt = parser.accepts("help", "Print this message.") - - val options = parser.parse(args : _*) - - if (options.has(helpOpt)) { - parser.printHelpOn(System.out) - Exit.exit(0) - } - - val commitIntervalMs = options.valueOf(commitIntervalOpt).intValue() - val fetchIntervalMs = options.valueOf(fetchIntervalOpt).intValue() - val threadCount = options.valueOf(numThreadsOpt).intValue() - val partitionCount = options.valueOf(numPartitionsOpt).intValue() - val zookeeper = options.valueOf(zookeeperOpt) - val reportingIntervalMs = options.valueOf(reportingIntervalOpt).intValue() - println("Commit thread count: %d; Partition count: %d, Commit interval: %d ms; Fetch interval: %d ms; Reporting interval: %d ms" - .format(threadCount, partitionCount, commitIntervalMs, fetchIntervalMs, reportingIntervalMs)) - - var zkUtils: ZkUtils = null - var commitThreads: Seq[CommitThread] = Seq() - var fetchThread: FetchThread = null - var statsThread: StatsThread = null - try { - zkUtils = ZkUtils(zookeeper, 6000, 2000, false) - commitThreads = (0 until threadCount).map { threadId => - new CommitThread(threadId, partitionCount, commitIntervalMs, zkUtils) - } - - fetchThread = new FetchThread(threadCount, fetchIntervalMs, zkUtils) - statsThread = new StatsThread(reportingIntervalMs, commitThreads, fetchThread) - - Runtime.getRuntime.addShutdownHook(new Thread() { - override def run() { - cleanShutdown() - statsThread.printStats() - } - }) - - commitThreads.foreach(_.start()) - - fetchThread.start() - - statsThread.start() - - commitThreads.foreach(_.join()) - fetchThread.join() - statsThread.join() - } - catch { - case e: Throwable => - println("Error: ", e) - } - finally { - cleanShutdown() - } - - def cleanShutdown() { - commitThreads.foreach(_.shutdown()) - commitThreads.foreach(_.join()) - if (fetchThread != null) { - fetchThread.shutdown() - fetchThread.join() - } - if (statsThread != null) { - statsThread.shutdown() - statsThread.join() - } - zkUtils.close() - } - - } -} - diff --git a/core/src/test/scala/unit/kafka/admin/AclCommandTest.scala b/core/src/test/scala/unit/kafka/admin/AclCommandTest.scala index 71754ba263318..73c5266d44f99 100644 --- a/core/src/test/scala/unit/kafka/admin/AclCommandTest.scala +++ b/core/src/test/scala/unit/kafka/admin/AclCommandTest.scala @@ -23,8 +23,9 @@ import kafka.security.auth._ import kafka.server.KafkaConfig import kafka.utils.{Logging, TestUtils} import kafka.zk.ZooKeeperTestHarness +import org.apache.kafka.common.resource.PatternType.{LITERAL, PREFIXED} import org.apache.kafka.common.security.auth.KafkaPrincipal -import org.junit.{After, Before, Test} +import org.junit.{Before, Test} class AclCommandTest extends ZooKeeperTestHarness with Logging { @@ -36,10 +37,10 @@ class AclCommandTest extends ZooKeeperTestHarness with Logging { private val AllowHostCommand = Array("--allow-host", "host1", "--allow-host", "host2") private val DenyHostCommand = Array("--deny-host", "host1", "--deny-host", "host2") - private val TopicResources = Set(Resource(Topic, "test-1", Literal), Resource(Topic, "test-2", Literal)) - private val GroupResources = Set(Resource(Group, "testGroup-1", Literal), Resource(Group, "testGroup-2", Literal)) - private val TransactionalIdResources = Set(Resource(TransactionalId, "t0", Literal), Resource(TransactionalId, "t1", Literal)) - private val TokenResources = Set(Resource(DelegationToken, "token1", Literal), Resource(DelegationToken, "token2", Literal)) + private val TopicResources = Set(Resource(Topic, "test-1", LITERAL), Resource(Topic, "test-2", LITERAL)) + private val GroupResources = Set(Resource(Group, "testGroup-1", LITERAL), Resource(Group, "testGroup-2", LITERAL)) + private val TransactionalIdResources = Set(Resource(TransactionalId, "t0", LITERAL), Resource(TransactionalId, "t1", LITERAL)) + private val TokenResources = Set(Resource(DelegationToken, "token1", LITERAL), Resource(DelegationToken, "token2", LITERAL)) private val ResourceToCommand = Map[Set[Resource], Array[String]]( TopicResources -> Array("--topic", "test-1", "--topic", "test-2"), @@ -50,13 +51,13 @@ class AclCommandTest extends ZooKeeperTestHarness with Logging { ) private val ResourceToOperations = Map[Set[Resource], (Set[Operation], Array[String])]( - TopicResources -> (Set(Read, Write, Create, Describe, Delete, DescribeConfigs, AlterConfigs), + TopicResources -> (Set(Read, Write, Create, Describe, Delete, DescribeConfigs, AlterConfigs, Alter), Array("--operation", "Read" , "--operation", "Write", "--operation", "Create", "--operation", "Describe", "--operation", "Delete", - "--operation", "DescribeConfigs", "--operation", "AlterConfigs")), - Set(Resource.ClusterResource) -> (Set(Create, ClusterAction, DescribeConfigs, AlterConfigs, IdempotentWrite), + "--operation", "DescribeConfigs", "--operation", "AlterConfigs", "--operation", "Alter")), + Set(Resource.ClusterResource) -> (Set(Create, ClusterAction, DescribeConfigs, AlterConfigs, IdempotentWrite, Alter, Describe), Array("--operation", "Create", "--operation", "ClusterAction", "--operation", "DescribeConfigs", - "--operation", "AlterConfigs", "--operation", "IdempotentWrite")), - GroupResources -> (Set(Read, Describe), Array("--operation", "Read", "--operation", "Describe")), + "--operation", "AlterConfigs", "--operation", "IdempotentWrite", "--operation", "Alter", "--operation", "Describe")), + GroupResources -> (Set(Read, Describe, Delete), Array("--operation", "Read", "--operation", "Describe", "--operation", "Delete")), TransactionalIdResources -> (Set(Describe, Write), Array("--operation", "Describe", "--operation", "Write")), TokenResources -> (Set(Describe), Array("--operation", "Describe")) ) @@ -64,7 +65,7 @@ class AclCommandTest extends ZooKeeperTestHarness with Logging { private def ProducerResourceToAcls(enableIdempotence: Boolean = false) = Map[Set[Resource], Set[Acl]]( TopicResources -> AclCommand.getAcls(Users, Allow, Set(Write, Describe, Create), Hosts), TransactionalIdResources -> AclCommand.getAcls(Users, Allow, Set(Write, Describe), Hosts), - Set(Resource.ClusterResource) -> AclCommand.getAcls(Users, Allow, + Set(Resource.ClusterResource) -> AclCommand.getAcls(Users, Allow, Set(if (enableIdempotence) Some(IdempotentWrite) else None).flatten, Hosts) ) @@ -132,7 +133,7 @@ class AclCommandTest extends ZooKeeperTestHarness with Logging { @Test def testAclsOnPrefixedResources(): Unit = { - val cmd = Array("--allow-principal", principal.toString, "--producer", "--topic", "Test-", "--resource-name-type", "Prefixed") + val cmd = Array("--allow-principal", principal.toString, "--producer", "--topic", "Test-", "--resource-pattern-type", "Prefixed") AclCommand.main(zkArgs ++ cmd :+ "--add") @@ -140,14 +141,14 @@ class AclCommandTest extends ZooKeeperTestHarness with Logging { val writeAcl = Acl(principal, Allow, Acl.WildCardHost, Write) val describeAcl = Acl(principal, Allow, Acl.WildCardHost, Describe) val createAcl = Acl(principal, Allow, Acl.WildCardHost, Create) - TestUtils.waitAndVerifyAcls(Set(writeAcl, describeAcl, createAcl), authorizer, Resource(Topic, "Test-", Prefixed)) + TestUtils.waitAndVerifyAcls(Set(writeAcl, describeAcl, createAcl), authorizer, Resource(Topic, "Test-", PREFIXED)) } AclCommand.main(zkArgs ++ cmd :+ "--remove" :+ "--force") withAuthorizer() { authorizer => - TestUtils.waitAndVerifyAcls(Set.empty[Acl], authorizer, Resource(Cluster, "kafka-cluster", Literal)) - TestUtils.waitAndVerifyAcls(Set.empty[Acl], authorizer, Resource(Topic, "Test-", Prefixed)) + TestUtils.waitAndVerifyAcls(Set.empty[Acl], authorizer, Resource(Cluster, "kafka-cluster", LITERAL)) + TestUtils.waitAndVerifyAcls(Set.empty[Acl], authorizer, Resource(Topic, "Test-", PREFIXED)) } } diff --git a/core/src/test/scala/unit/kafka/admin/AddPartitionsTest.scala b/core/src/test/scala/unit/kafka/admin/AddPartitionsTest.scala index 43d8ec8ea5bcb..4d1e4abb9b855 100755 --- a/core/src/test/scala/unit/kafka/admin/AddPartitionsTest.scala +++ b/core/src/test/scala/unit/kafka/admin/AddPartitionsTest.scala @@ -17,24 +17,23 @@ package kafka.admin -import kafka.api.TopicMetadata +import kafka.network.SocketServer import org.junit.Assert._ -import kafka.zk.ZooKeeperTestHarness import kafka.utils.TestUtils._ import kafka.utils.TestUtils -import kafka.cluster.Broker -import kafka.client.ClientUtils -import kafka.server.{KafkaConfig, KafkaServer} +import kafka.server.BaseRequestTest import org.apache.kafka.common.TopicPartition import org.apache.kafka.common.errors.InvalidReplicaAssignmentException -import org.apache.kafka.common.network.ListenerName -import org.apache.kafka.common.security.auth.SecurityProtocol -import org.junit.{After, Before, Test} +import org.apache.kafka.common.protocol.ApiKeys +import org.apache.kafka.common.requests.MetadataResponse.TopicMetadata +import org.apache.kafka.common.requests.{MetadataRequest, MetadataResponse} +import org.junit.{Before, Test} -class AddPartitionsTest extends ZooKeeperTestHarness { - var configs: Seq[KafkaConfig] = null - var servers: Seq[KafkaServer] = Seq.empty[KafkaServer] - var brokers: Seq[Broker] = Seq.empty[Broker] +import scala.collection.JavaConverters._ + +class AddPartitionsTest extends BaseRequestTest { + + protected override def numBrokers: Int = 4 val partitionId = 0 @@ -53,22 +52,10 @@ class AddPartitionsTest extends ZooKeeperTestHarness { override def setUp() { super.setUp() - 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 => TestUtils.createBroker(s.config.brokerId, s.config.hostName, TestUtils.boundPort(s))) - - // create topics first - createTopic(zkClient, topic1, partitionReplicaAssignment = topic1Assignment, servers = servers) - createTopic(zkClient, topic2, partitionReplicaAssignment = topic2Assignment, servers = servers) - createTopic(zkClient, topic3, partitionReplicaAssignment = topic3Assignment, servers = servers) - createTopic(zkClient, topic4, partitionReplicaAssignment = topic4Assignment, servers = servers) - } - - @After - override def tearDown() { - TestUtils.shutdownServers(servers) - super.tearDown() + createTopic(topic1, partitionReplicaAssignment = topic1Assignment) + createTopic(topic2, partitionReplicaAssignment = topic2Assignment) + createTopic(topic3, partitionReplicaAssignment = topic3Assignment) + createTopic(topic4, partitionReplicaAssignment = topic4Assignment) } @Test @@ -108,17 +95,15 @@ class AddPartitionsTest extends ZooKeeperTestHarness { // read metadata from a broker and verify the new topic partitions exist TestUtils.waitUntilMetadataIsPropagated(servers, topic1, 1) TestUtils.waitUntilMetadataIsPropagated(servers, topic1, 2) - val listenerName = ListenerName.forSecurityProtocol(SecurityProtocol.PLAINTEXT) - val metadata = ClientUtils.fetchTopicMetadata(Set(topic1), brokers.map(_.brokerEndPoint(listenerName)), - "AddPartitionsTest-testIncrementPartitions", 2000, 0).topicsMetadata - val metaDataForTopic1 = metadata.filter(p => p.topic.equals(topic1)) - val partitionDataForTopic1 = metaDataForTopic1.head.partitionsMetadata.sortBy(_.partitionId) - assertEquals(partitionDataForTopic1.size, 3) - assertEquals(partitionDataForTopic1(1).partitionId, 1) - assertEquals(partitionDataForTopic1(2).partitionId, 2) - val replicas = partitionDataForTopic1(1).replicas + val response = sendMetadataRequest(new MetadataRequest.Builder(Seq(topic1).asJava, false).build) + assertEquals(1, response.topicMetadata.size) + val partitions = response.topicMetadata.asScala.head.partitionMetadata.asScala.sortBy(_.partition) + assertEquals(partitions.size, 3) + assertEquals(1, partitions(1).partition) + assertEquals(2, partitions(2).partition) + val replicas = partitions(1).replicas assertEquals(replicas.size, 2) - assert(replicas.contains(partitionDataForTopic1(1).leader.get)) + assertTrue(replicas.contains(partitions(1).leader)) } @Test @@ -137,18 +122,18 @@ class AddPartitionsTest extends 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.map(_.brokerEndPoint(ListenerName.forSecurityProtocol(SecurityProtocol.PLAINTEXT))), - "AddPartitionsTest-testManualAssignmentOfReplicas", 2000, 0).topicsMetadata - val metaDataForTopic2 = metadata.filter(_.topic == topic2) - val partitionDataForTopic2 = metaDataForTopic2.head.partitionsMetadata.sortBy(_.partitionId) - assertEquals(3, partitionDataForTopic2.size) - assertEquals(1, partitionDataForTopic2(1).partitionId) - assertEquals(2, partitionDataForTopic2(2).partitionId) - val replicas = partitionDataForTopic2(1).replicas + val response = sendMetadataRequest(new MetadataRequest.Builder(Seq(topic2).asJava, false).build) + assertEquals(1, response.topicMetadata.size) + val topicMetadata = response.topicMetadata.asScala.head + val partitionMetadata = topicMetadata.partitionMetadata.asScala.sortBy(_.partition) + assertEquals(3, topicMetadata.partitionMetadata.size) + assertEquals(0, partitionMetadata(0).partition) + assertEquals(1, partitionMetadata(1).partition) + assertEquals(2, partitionMetadata(2).partition) + val replicas = topicMetadata.partitionMetadata.get(1).replicas assertEquals(2, replicas.size) - assertTrue(replicas.head.id == 0 || replicas.head.id == 1) - assertTrue(replicas(1).id == 0 || replicas(1).id == 1) + assertTrue(replicas.asScala.head.id == 0 || replicas.asScala.head.id == 1) + assertTrue(replicas.asScala(1).id == 0 || replicas.asScala(1).id == 1) } @Test @@ -163,19 +148,16 @@ class AddPartitionsTest extends ZooKeeperTestHarness { TestUtils.waitUntilMetadataIsPropagated(servers, topic3, 5) TestUtils.waitUntilMetadataIsPropagated(servers, topic3, 6) - val metadata = ClientUtils.fetchTopicMetadata(Set(topic3), - brokers.map(_.brokerEndPoint(ListenerName.forSecurityProtocol(SecurityProtocol.PLAINTEXT))), - "AddPartitionsTest-testReplicaPlacementAllServers", 2000, 0).topicsMetadata - - val metaDataForTopic3 = metadata.find(p => p.topic == topic3).get - - validateLeaderAndReplicas(metaDataForTopic3, 0, 2, Set(2, 3, 0, 1)) - validateLeaderAndReplicas(metaDataForTopic3, 1, 3, Set(3, 2, 0, 1)) - validateLeaderAndReplicas(metaDataForTopic3, 2, 0, Set(0, 3, 1, 2)) - validateLeaderAndReplicas(metaDataForTopic3, 3, 1, Set(1, 0, 2, 3)) - validateLeaderAndReplicas(metaDataForTopic3, 4, 2, Set(2, 3, 0, 1)) - validateLeaderAndReplicas(metaDataForTopic3, 5, 3, Set(3, 0, 1, 2)) - validateLeaderAndReplicas(metaDataForTopic3, 6, 0, Set(0, 1, 2, 3)) + val response = sendMetadataRequest(new MetadataRequest.Builder(Seq(topic3).asJava, false).build) + assertEquals(1, response.topicMetadata.size) + val topicMetadata = response.topicMetadata.asScala.head + validateLeaderAndReplicas(topicMetadata, 0, 2, Set(2, 3, 0, 1)) + validateLeaderAndReplicas(topicMetadata, 1, 3, Set(3, 2, 0, 1)) + validateLeaderAndReplicas(topicMetadata, 2, 0, Set(0, 3, 1, 2)) + validateLeaderAndReplicas(topicMetadata, 3, 1, Set(1, 0, 2, 3)) + validateLeaderAndReplicas(topicMetadata, 4, 2, Set(2, 3, 0, 1)) + validateLeaderAndReplicas(topicMetadata, 5, 3, Set(3, 0, 1, 2)) + validateLeaderAndReplicas(topicMetadata, 6, 0, Set(0, 1, 2, 3)) } @Test @@ -186,25 +168,27 @@ class AddPartitionsTest extends ZooKeeperTestHarness { TestUtils.waitUntilMetadataIsPropagated(servers, topic2, 1) TestUtils.waitUntilMetadataIsPropagated(servers, topic2, 2) - val metadata = ClientUtils.fetchTopicMetadata(Set(topic2), - brokers.map(_.brokerEndPoint(ListenerName.forSecurityProtocol(SecurityProtocol.PLAINTEXT))), - "AddPartitionsTest-testReplicaPlacementPartialServers", 2000, 0).topicsMetadata - - val metaDataForTopic2 = metadata.find(p => p.topic == topic2).get - - validateLeaderAndReplicas(metaDataForTopic2, 0, 1, Set(1, 2)) - validateLeaderAndReplicas(metaDataForTopic2, 1, 2, Set(0, 2)) - validateLeaderAndReplicas(metaDataForTopic2, 2, 3, Set(1, 3)) + val response = sendMetadataRequest(new MetadataRequest.Builder(Seq(topic2).asJava, false).build) + assertEquals(1, response.topicMetadata.size) + val topicMetadata = response.topicMetadata.asScala.head + validateLeaderAndReplicas(topicMetadata, 0, 1, Set(1, 2)) + validateLeaderAndReplicas(topicMetadata, 1, 2, Set(0, 2)) + validateLeaderAndReplicas(topicMetadata, 2, 3, Set(1, 3)) } - def validateLeaderAndReplicas(metadata: TopicMetadata, partitionId: Int, expectedLeaderId: Int, expectedReplicas: Set[Int]) = { - val partitionOpt = metadata.partitionsMetadata.find(_.partitionId == partitionId) + def validateLeaderAndReplicas(metadata: TopicMetadata, partitionId: Int, expectedLeaderId: Int, + expectedReplicas: Set[Int]): Unit = { + val partitionOpt = metadata.partitionMetadata.asScala.find(_.partition == partitionId) assertTrue(s"Partition $partitionId should exist", partitionOpt.isDefined) val partition = partitionOpt.get - assertTrue("Partition leader should exist", partition.leader.isDefined) - assertEquals("Partition leader id should match", expectedLeaderId, partition.leader.get.id) + assertNotNull("Partition leader should exist", partition.leader) + assertEquals("Partition leader id should match", expectedLeaderId, partition.leaderId) + assertEquals("Replica set should match", expectedReplicas, partition.replicas.asScala.map(_.id).toSet) + } - assertEquals("Replica set should match", expectedReplicas, partition.replicas.map(_.id).toSet) + private def sendMetadataRequest(request: MetadataRequest, destination: Option[SocketServer] = None): MetadataResponse = { + val response = connectAndSend(request, ApiKeys.METADATA, destination = destination.getOrElse(anySocketServer)) + MetadataResponse.parse(response, request.version) } } diff --git a/core/src/test/scala/unit/kafka/admin/ConfigCommandTest.scala b/core/src/test/scala/unit/kafka/admin/ConfigCommandTest.scala index a24800f3cea7b..2e8179ce44c34 100644 --- a/core/src/test/scala/unit/kafka/admin/ConfigCommandTest.scala +++ b/core/src/test/scala/unit/kafka/admin/ConfigCommandTest.scala @@ -20,21 +20,25 @@ import java.util import java.util.Properties import kafka.admin.ConfigCommand.ConfigCommandOptions -import kafka.common.InvalidConfigException -import kafka.server.ConfigEntityName +import kafka.api.ApiVersion +import kafka.cluster.{Broker, EndPoint} +import kafka.server.{ConfigEntityName, KafkaConfig} import kafka.utils.{Exit, Logging} -import kafka.zk.{AdminZkClient, KafkaZkClient, ZooKeeperTestHarness} +import kafka.zk.{AdminZkClient, BrokerInfo, KafkaZkClient, ZooKeeperTestHarness} import org.apache.kafka.clients.admin._ -import org.apache.kafka.common.config.ConfigResource +import org.apache.kafka.common.config.{ConfigException, ConfigResource} import org.apache.kafka.common.internals.KafkaFutureImpl import org.apache.kafka.common.Node +import org.apache.kafka.common.errors.InvalidConfigurationException +import org.apache.kafka.common.network.ListenerName +import org.apache.kafka.common.security.auth.SecurityProtocol import org.apache.kafka.common.security.scram.internals.ScramCredentialUtils import org.apache.kafka.common.utils.Sanitizer import org.easymock.EasyMock import org.junit.Assert._ import org.junit.Test -import scala.collection.mutable +import scala.collection.{Seq, mutable} import scala.collection.JavaConverters._ class ConfigCommandTest extends ZooKeeperTestHarness with Logging { @@ -51,7 +55,7 @@ class ConfigCommandTest extends ZooKeeperTestHarness with Logging { "--entity-name", "1", "--entity-type", "brokers", "--alter", - "--add-config", "message.max.size=100000")) + "--add-config", "security.inter.broker.protocol=PLAINTEXT")) } @Test @@ -306,14 +310,99 @@ class ConfigCommandTest extends ZooKeeperTestHarness with Logging { ConfigCommand.alterConfig(null, createOpts, new DummyAdminZkClient(zkClient)) } - @Test (expected = classOf[IllegalArgumentException]) - def shouldNotUpdateDynamicBrokerConfigUsingZooKeeper(): Unit = { - val createOpts = new ConfigCommandOptions(Array("--zookeeper", zkConnect, - "--entity-name", "1", - "--entity-type", "brokers", - "--alter", - "--add-config", "message.max.size=100000")) - ConfigCommand.alterConfig(null, createOpts, new DummyAdminZkClient(zkClient)) + @Test + def testDynamicBrokerConfigUpdateUsingZooKeeper(): Unit = { + val brokerId = "1" + val adminZkClient = new AdminZkClient(zkClient) + val alterOpts = Array("--zookeeper", zkConnect, "--entity-type", "brokers", "--alter") + + def entityOpt(brokerId: Option[String]): Array[String] = { + brokerId.map(id => Array("--entity-name", id)).getOrElse(Array("--entity-default")) + } + + def alterConfig(configs: Map[String, String], brokerId: Option[String], + encoderConfigs: Map[String, String] = Map.empty): Unit = { + val configStr = (configs ++ encoderConfigs).map { case (k, v) => s"$k=$v" }.mkString(",") + val addOpts = new ConfigCommandOptions(alterOpts ++ entityOpt(brokerId) ++ Array("--add-config", configStr)) + ConfigCommand.alterConfig(zkClient, addOpts, adminZkClient) + } + + def verifyConfig(configs: Map[String, String], brokerId: Option[String]): Unit = { + val entityConfigs = zkClient.getEntityConfigs("brokers", brokerId.getOrElse(ConfigEntityName.Default)) + assertEquals(configs, entityConfigs.asScala) + } + + def alterAndVerifyConfig(configs: Map[String, String], brokerId: Option[String]): Unit = { + alterConfig(configs, brokerId) + verifyConfig(configs, brokerId) + } + + def deleteAndVerifyConfig(configNames: Set[String], brokerId: Option[String]): Unit = { + val deleteOpts = new ConfigCommandOptions(alterOpts ++ entityOpt(brokerId) ++ + Array("--delete-config", configNames.mkString(","))) + ConfigCommand.alterConfig(zkClient, deleteOpts, adminZkClient) + verifyConfig(Map.empty, brokerId) + } + + // Add config + alterAndVerifyConfig(Map("message.max.size" -> "110000"), Some(brokerId)) + alterAndVerifyConfig(Map("message.max.size" -> "120000"), None) + + // Change config + alterAndVerifyConfig(Map("message.max.size" -> "130000"), Some(brokerId)) + alterAndVerifyConfig(Map("message.max.size" -> "140000"), None) + + // Delete config + deleteAndVerifyConfig(Set("message.max.size"), Some(brokerId)) + deleteAndVerifyConfig(Set("message.max.size"), None) + + // Listener configs: should work only with listener name + alterAndVerifyConfig(Map("listener.name.external.ssl.keystore.location" -> "/tmp/test.jks"), Some(brokerId)) + intercept[ConfigException](alterConfig(Map("ssl.keystore.location" -> "/tmp/test.jks"), Some(brokerId))) + + // Per-broker config configured at default cluster-level should fail + intercept[ConfigException](alterConfig(Map("listener.name.external.ssl.keystore.location" -> "/tmp/test.jks"), None)) + deleteAndVerifyConfig(Set("listener.name.external.ssl.keystore.location"), Some(brokerId)) + + // Password config update without encoder secret should fail + intercept[IllegalArgumentException](alterConfig(Map("listener.name.external.ssl.keystore.password" -> "secret"), Some(brokerId))) + + // Password config update with encoder secret should succeed and encoded password must be stored in ZK + val configs = Map("listener.name.external.ssl.keystore.password" -> "secret", "log.cleaner.threads" -> "2") + val encoderConfigs = Map(KafkaConfig.PasswordEncoderSecretProp -> "encoder-secret") + alterConfig(configs, Some(brokerId), encoderConfigs) + val brokerConfigs = zkClient.getEntityConfigs("brokers", brokerId) + assertFalse("Encoder secret stored in ZooKeeper", brokerConfigs.contains(KafkaConfig.PasswordEncoderSecretProp)) + assertEquals("2", brokerConfigs.getProperty("log.cleaner.threads")) // not encoded + val encodedPassword = brokerConfigs.getProperty("listener.name.external.ssl.keystore.password") + val passwordEncoder = ConfigCommand.createPasswordEncoder(encoderConfigs) + assertEquals("secret", passwordEncoder.decode(encodedPassword).value) + assertEquals(configs.size, brokerConfigs.size) + + // Password config update with overrides for encoder parameters + val configs2 = Map("listener.name.internal.ssl.keystore.password" -> "secret2") + val encoderConfigs2 = Map(KafkaConfig.PasswordEncoderSecretProp -> "encoder-secret", + KafkaConfig.PasswordEncoderCipherAlgorithmProp -> "DES/CBC/PKCS5Padding", + KafkaConfig.PasswordEncoderIterationsProp -> "1024", + KafkaConfig.PasswordEncoderKeyFactoryAlgorithmProp -> "PBKDF2WithHmacSHA1", + KafkaConfig.PasswordEncoderKeyLengthProp -> "64") + alterConfig(configs2, Some(brokerId), encoderConfigs2) + val brokerConfigs2 = zkClient.getEntityConfigs("brokers", brokerId) + val encodedPassword2 = brokerConfigs2.getProperty("listener.name.internal.ssl.keystore.password") + assertEquals("secret2", ConfigCommand.createPasswordEncoder(encoderConfigs).decode(encodedPassword2).value) + assertEquals("secret2", ConfigCommand.createPasswordEncoder(encoderConfigs2).decode(encodedPassword2).value) + + + // Password config update at default cluster-level should fail + intercept[ConfigException](alterConfig(configs, None, encoderConfigs)) + + // Dynamic config updates using ZK should fail if broker is running. + registerBrokerInZk(brokerId.toInt) + intercept[IllegalArgumentException](alterConfig(Map("message.max.size" -> "210000"), Some(brokerId))) + intercept[IllegalArgumentException](alterConfig(Map("message.max.size" -> "220000"), None)) + + // Dynamic config updates using ZK should for a different broker that is not running should succeed + alterAndVerifyConfig(Map("message.max.size" -> "230000"), Some("2")) } @Test (expected = classOf[IllegalArgumentException]) @@ -322,7 +411,7 @@ class ConfigCommandTest extends ZooKeeperTestHarness with Logging { "--entity-name", "1", "--entity-type", "brokers", "--alter", - "--add-config", "a=")) + "--add-config", "a==")) ConfigCommand.alterConfig(null, createOpts, new DummyAdminZkClient(zkClient)) } @@ -336,7 +425,7 @@ class ConfigCommandTest extends ZooKeeperTestHarness with Logging { ConfigCommand.alterConfig(null, createOpts, new DummyAdminZkClient(zkClient)) } - @Test (expected = classOf[InvalidConfigException]) + @Test (expected = classOf[InvalidConfigurationException]) def shouldNotUpdateBrokerConfigIfNonExistingConfigIsDeleted(): Unit = { val createOpts = new ConfigCommandOptions(Array("--zookeeper", zkConnect, "--entity-name", "my-topic", @@ -593,6 +682,14 @@ class ConfigCommandTest extends ZooKeeperTestHarness with Logging { Seq("/clients/client-3", sanitizedPrincipal + "/clients/client-2")) } + private def registerBrokerInZk(id: Int): Unit = { + zkClient.createTopLevelPaths() + val securityProtocol = SecurityProtocol.PLAINTEXT + val endpoint = new EndPoint("localhost", 9092, ListenerName.forSecurityProtocol(securityProtocol), securityProtocol) + val brokerInfo = BrokerInfo(Broker(id, Seq(endpoint), rack = None), ApiVersion.latestVersion, jmxPort = 9192) + zkClient.registerBrokerInZk(brokerInfo) + } + class DummyAdminZkClient(zkClient: KafkaZkClient) extends AdminZkClient(zkClient) { override def changeBrokerConfig(brokerIds: Seq[Int], configs: Properties): Unit = {} override def fetchEntityConfig(entityType: String, entityName: String): Properties = {new Properties} diff --git a/core/src/test/scala/unit/kafka/admin/ConsumerGroupCommandTest.scala b/core/src/test/scala/unit/kafka/admin/ConsumerGroupCommandTest.scala index 7c451093e676a..2bed673ee7390 100644 --- a/core/src/test/scala/unit/kafka/admin/ConsumerGroupCommandTest.scala +++ b/core/src/test/scala/unit/kafka/admin/ConsumerGroupCommandTest.scala @@ -20,8 +20,7 @@ package kafka.admin import java.util.concurrent.{ExecutorService, Executors, TimeUnit} import java.util.{Collections, Properties} -import kafka.admin.ConsumerGroupCommand.{ConsumerGroupCommandOptions, ConsumerGroupService, KafkaConsumerGroupService, ZkConsumerGroupService} -import kafka.consumer.{OldConsumer, Whitelist} +import kafka.admin.ConsumerGroupCommand.{ConsumerGroupCommandOptions, ConsumerGroupService} import kafka.integration.KafkaServerTestHarness import kafka.server.KafkaConfig import kafka.utils.TestUtils @@ -40,8 +39,6 @@ class ConsumerGroupCommandTest extends KafkaServerTestHarness { val topic = "foo" val group = "test.group" - @deprecated("This field will be removed in a future release", "0.11.0.0") - private val oldConsumers = new ArrayBuffer[OldConsumer] private var consumerGroupService: List[ConsumerGroupService] = List() private var consumerGroupExecutors: List[AbstractConsumerGroupExecutor] = List() @@ -62,18 +59,9 @@ class ConsumerGroupCommandTest extends KafkaServerTestHarness { override def tearDown(): Unit = { consumerGroupService.foreach(_.close()) consumerGroupExecutors.foreach(_.shutdown()) - oldConsumers.foreach(_.stop()) super.tearDown() } - @deprecated("This test has been deprecated and will be removed in a future release.", "0.11.1.0") - def createOldConsumer(): Unit = { - val consumerProps = new Properties - consumerProps.setProperty("group.id", group) - consumerProps.setProperty("zookeeper.connect", zkConnect) - oldConsumers += new OldConsumer(Whitelist(topic), consumerProps) - } - def committedOffsets(topic: String = topic, group: String = group): Map[TopicPartition, Long] = { val props = new Properties props.put("bootstrap.servers", brokerList) @@ -93,13 +81,9 @@ class ConsumerGroupCommandTest extends KafkaServerTestHarness { } } - def stopRandomOldConsumer(): Unit = { - oldConsumers.head.stop() - } - def getConsumerGroupService(args: Array[String]): ConsumerGroupService = { val opts = new ConsumerGroupCommandOptions(args) - val service = if (opts.useOldConsumer) new ZkConsumerGroupService(opts) else new KafkaConsumerGroupService(opts) + val service = new ConsumerGroupService(opts) consumerGroupService = service :: consumerGroupService service } diff --git a/core/src/test/scala/unit/kafka/admin/DeleteConsumerGroupTest.scala b/core/src/test/scala/unit/kafka/admin/DeleteConsumerGroupTest.scala deleted file mode 100644 index da17f168f2df9..0000000000000 --- a/core/src/test/scala/unit/kafka/admin/DeleteConsumerGroupTest.scala +++ /dev/null @@ -1,236 +0,0 @@ -/** - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package kafka.admin - -import java.nio.charset.StandardCharsets - -import kafka.utils._ -import kafka.server.KafkaConfig -import org.junit.{After, Before, Test} -import kafka.consumer._ -import org.apache.kafka.clients.producer.{KafkaProducer, ProducerRecord} -import kafka.integration.KafkaServerTestHarness -import org.apache.kafka.common.security.JaasUtils - - -@deprecated("This test has been deprecated and will be removed in a future release.", "0.11.0.0") -class DeleteConsumerGroupTest extends KafkaServerTestHarness { - def generateConfigs = TestUtils.createBrokerConfigs(3, zkConnect, false, true).map(KafkaConfig.fromProps) - var zkUtils: ZkUtils = null - - @Before - override def setUp() { - super.setUp() - zkUtils = ZkUtils(zkConnect, zkSessionTimeout, zkConnectionTimeout, zkAclsEnabled.getOrElse(JaasUtils.isZkSecurityEnabled)) - } - - @After - override def tearDown() { - if (zkUtils != null) - CoreUtils.swallow(zkUtils.close(), this) - super.tearDown() - } - - - @Test - def testGroupWideDeleteInZK() { - val topic = "test" - val groupToDelete = "groupToDelete" - val otherGroup = "otherGroup" - - createTopic(topic, 1, 3) - fillInConsumerGroupInfo(topic, groupToDelete, "consumer", 0, 10, false) - fillInConsumerGroupInfo(topic, otherGroup, "consumer", 0, 10, false) - - AdminUtils.deleteConsumerGroupInZK(zkUtils, 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" - - createTopic(topic, 1, 3) - fillInConsumerGroupInfo(topic, groupToDelete, "consumer", 0, 10, true) - fillInConsumerGroupInfo(topic, otherGroup, "consumer", 0, 10, false) - - AdminUtils.deleteConsumerGroupInZK(zkUtils, 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" - createTopic(topic, 1, 3) - fillInConsumerGroupInfo(topic, groupToDelete, "consumer", 0, 10, false) - fillInConsumerGroupInfo(topic, otherGroup, "consumer", 0, 10, false) - - AdminUtils.deleteConsumerGroupInfoForTopicInZK(zkUtils, 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" - createTopic(topicToDelete, 1, 3) - createTopic(otherTopic, 1, 3) - - fillInConsumerGroupInfo(topicToDelete, groupToDelete, "consumer", 0, 10, false) - fillInConsumerGroupInfo(otherTopic, groupToDelete, "consumer", 0, 10, false) - fillInConsumerGroupInfo(topicToDelete, otherGroup, "consumer", 0, 10, false) - - AdminUtils.deleteConsumerGroupInfoForTopicInZK(zkUtils, 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" - createTopic(topicToDelete, 1, 3) - createTopic(otherTopic, 1, 3) - - fillInConsumerGroupInfo(topicToDelete, group, "consumer", 0, 10, true) - fillInConsumerGroupInfo(otherTopic, group, "consumer", 0, 10, true) - - AdminUtils.deleteConsumerGroupInfoForTopicInZK(zkUtils, 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") - - createTopic(topicToDelete, 1, 3) - createTopic(otherTopic, 1, 3) - 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(zkUtils, topicToDelete) - - TestUtils.waitUntilTrue(() => !groupTopicDirsForTopicToDelete.exists(groupTopicOffsetAndOwnerDirsExist), - "Consumer group info on deleted 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" - - createTopic(topic, 1, 3) - val dir = new ZKGroupTopicDirs(group, topic) - fillInConsumerGroupInfo(topic, dir.group, "consumer", 0, 10, false) - - AdminUtils.deleteTopic(zkUtils, topic) - TestUtils.verifyTopicDeletion(zkClient, topic, 1, servers) - AdminUtils.deleteAllConsumerGroupInfoForTopicInZK(zkUtils, topic) - - TestUtils.waitUntilTrue(() => !groupDirExists(dir), - "Consumer group info on related topics should be deleted by DeleteAllConsumerGroupInfoForTopicInZK") - //produce events - val producer = TestUtils.createNewProducer(brokerList) - try { - produceEvents(producer, topic, List.fill(10)("test")) - } finally { - producer.close() - } - - //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) - try { - val messageStream = consumerConnector.createMessageStreams(Map(topic -> 1))(topic).head - consumeEvents(messageStream, 5) - consumerConnector.commitOffsets(false) - } finally { - 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(zkUtils.getConsumerPartitionOwnerPath(group, topic, partition), "") - zkUtils.makeSurePersistentPathExists(dir.consumerRegistryDir) - if (registerConsumer) { - zkUtils.createEphemeralPathExpectConflict(dir.consumerRegistryDir + "/" + consumerId, "") - } - } - - private def groupDirExists(dir: ZKGroupDirs) = { - zkUtils.pathExists(dir.consumerGroupDir) - } - - private def groupTopicOffsetAndOwnerDirsExist(dir: ZKGroupTopicDirs) = { - zkUtils.pathExists(dir.consumerOffsetDir) && zkUtils.pathExists(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(StandardCharsets.UTF_8)))) - } - - 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/DeleteConsumerGroupsTest.scala b/core/src/test/scala/unit/kafka/admin/DeleteConsumerGroupsTest.scala index ef3b17c393850..aae48d1d527f9 100644 --- a/core/src/test/scala/unit/kafka/admin/DeleteConsumerGroupsTest.scala +++ b/core/src/test/scala/unit/kafka/admin/DeleteConsumerGroupsTest.scala @@ -14,16 +14,15 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package unit.kafka.admin +package kafka.admin import joptsimple.OptionException -import kafka.admin.ConsumerGroupCommandTest import kafka.utils.TestUtils import org.apache.kafka.common.protocol.Errors import org.junit.Assert._ import org.junit.Test -class DeleteConsumerGroupTest extends ConsumerGroupCommandTest { +class DeleteConsumerGroupsTest extends ConsumerGroupCommandTest { @Test(expected = classOf[OptionException]) def testDeleteWithTopicOption() { diff --git a/core/src/test/scala/unit/kafka/admin/DescribeConsumerGroupTest.scala b/core/src/test/scala/unit/kafka/admin/DescribeConsumerGroupTest.scala index a2361b706a3bf..6c2b09b99f612 100644 --- a/core/src/test/scala/unit/kafka/admin/DescribeConsumerGroupTest.scala +++ b/core/src/test/scala/unit/kafka/admin/DescribeConsumerGroupTest.scala @@ -33,70 +33,6 @@ class DescribeConsumerGroupTest extends ConsumerGroupCommandTest { private val describeTypeState = Array(Array("--state")) private val describeTypes = describeTypeOffsets ++ describeTypeMembers ++ describeTypeState - @Test - @deprecated("This test has been deprecated and will be removed in a future release.", "0.11.0.0") - def testDescribeNonExistingGroupWithOldConsumer() { - TestUtils.createOffsetsTopic(zkClient, servers) - createOldConsumer() - val service = getConsumerGroupService(Array("--zookeeper", zkConnect, "--describe", "--group", "missing.group")) - TestUtils.waitUntilTrue(() => service.collectGroupOffsets()._2.isEmpty, "Expected no rows in describe group results.") - } - - @Test - @deprecated("This test has been deprecated and will be removed in a future release.", "0.11.0.0") - def testDescribeExistingGroupWithOldConsumer() { - TestUtils.createOffsetsTopic(zkClient, servers) - createOldConsumer() - val service = getConsumerGroupService(Array("--zookeeper", zkConnect, "--describe", "--group", group)) - - TestUtils.waitUntilTrue(() => { - val (_, assignments) = service.collectGroupOffsets() - assignments.isDefined && - assignments.get.count(_.group == group) == 1 && - assignments.get.filter(_.group == group).head.consumerId.exists(_.trim != ConsumerGroupCommand.MISSING_COLUMN_VALUE) - }, "Expected rows and a consumer id column in describe group results.") - } - - @Test - @deprecated("This test has been deprecated and will be removed in a future release.", "0.11.0.0") - def testDescribeExistingGroupWithNoMembersWithOldConsumer() { - TestUtils.createOffsetsTopic(zkClient, servers) - createOldConsumer() - val service = getConsumerGroupService(Array("--zookeeper", zkConnect, "--describe", "--group", group)) - - TestUtils.waitUntilTrue(() => { - val (_, assignments) = service.collectGroupOffsets() - assignments.isDefined && - assignments.get.count(_.group == group) == 1 && - assignments.get.filter(_.group == group).head.consumerId.exists(_.trim != ConsumerGroupCommand.MISSING_COLUMN_VALUE) - }, "Expected rows and a consumer id column in describe group results.") - stopRandomOldConsumer() - - TestUtils.waitUntilTrue(() => { - val (_, assignments) = service.collectGroupOffsets() - assignments.isDefined && - assignments.get.count(_.group == group) == 1 && - assignments.get.filter(_.group == group).head.consumerId.exists(_.trim == ConsumerGroupCommand.MISSING_COLUMN_VALUE) // the member should be gone - }, "Expected no active member in describe group results.") - } - - @Test - @deprecated("This test has been deprecated and will be removed in a future release.", "0.11.0.0") - def testDescribeConsumersWithNoAssignedPartitionsWithOldConsumer() { - TestUtils.createOffsetsTopic(zkClient, servers) - createOldConsumer() - createOldConsumer() - val service = getConsumerGroupService(Array("--zookeeper", zkConnect, "--describe", "--group", group)) - - TestUtils.waitUntilTrue(() => { - val (_, assignments) = service.collectGroupOffsets() - assignments.isDefined && - assignments.get.count(_.group == group) == 2 && - assignments.get.count { x => x.group == group && x.partition.isDefined } == 1 && - assignments.get.count { x => x.group == group && x.partition.isEmpty } == 1 - }, "Expected rows for consumers with no assigned partitions in describe group results.") - } - @Test def testDescribeNonExistingGroup() { TestUtils.createOffsetsTopic(zkClient, servers) diff --git a/core/src/test/scala/unit/kafka/admin/ListConsumerGroupTest.scala b/core/src/test/scala/unit/kafka/admin/ListConsumerGroupTest.scala index c83e00287970d..32f6614f9285a 100644 --- a/core/src/test/scala/unit/kafka/admin/ListConsumerGroupTest.scala +++ b/core/src/test/scala/unit/kafka/admin/ListConsumerGroupTest.scala @@ -16,60 +16,12 @@ */ package kafka.admin -import java.util.Properties - import joptsimple.OptionException import org.junit.Test -import kafka.admin.ConsumerGroupCommand.ConsumerGroupCommandOptions -import kafka.admin.ConsumerGroupCommand.ZkConsumerGroupService -import kafka.consumer.{OldConsumer, Whitelist} import kafka.utils.TestUtils -import org.easymock.EasyMock class ListConsumerGroupTest extends ConsumerGroupCommandTest { - @Test - def testListOldConsumerGroups() { - val topicFilter = Whitelist(topic) - val props = new Properties - props.setProperty("group.id", group) - props.setProperty("zookeeper.connect", zkConnect) - // mocks - val consumer1Mock = EasyMock.createMockBuilder(classOf[OldConsumer]).withConstructor(topicFilter, props).createMock() - props.setProperty("group.id", "some.other.group") - val consumer2Mock = EasyMock.createMockBuilder(classOf[OldConsumer]).withConstructor(topicFilter, props).createMock() - - // stubs - val opts = new ConsumerGroupCommandOptions(Array("--zookeeper", zkConnect)) - val consumerGroupCommand = new ZkConsumerGroupService(opts) - - // simulation - EasyMock.replay(consumer1Mock) - EasyMock.replay(consumer2Mock) - - // action/test - TestUtils.waitUntilTrue(() => { - val groups = consumerGroupCommand.listGroups() - groups.size == 2 && groups.contains(group) - }, "Expected a different list group results.") - - // cleanup - consumerGroupCommand.close() - consumer1Mock.stop() - consumer2Mock.stop() - } - - @Test - def testListGroupWithNoExistingGroup() { - val opts = new ConsumerGroupCommandOptions(Array("--zookeeper", zkConnect)) - val consumerGroupCommand = new ZkConsumerGroupService(opts) - try { - assert(consumerGroupCommand.listGroups().isEmpty) - } finally { - consumerGroupCommand.close() - } - } - @Test def testListConsumerGroups() { val simpleGroup = "simple-group" diff --git a/core/src/test/scala/unit/kafka/admin/ResetConsumerGroupOffsetTest.scala b/core/src/test/scala/unit/kafka/admin/ResetConsumerGroupOffsetTest.scala index 116b455cb87cf..3644cd7f42e68 100644 --- a/core/src/test/scala/unit/kafka/admin/ResetConsumerGroupOffsetTest.scala +++ b/core/src/test/scala/unit/kafka/admin/ResetConsumerGroupOffsetTest.scala @@ -92,7 +92,7 @@ class ResetConsumerGroupOffsetTest extends ConsumerGroupCommandTest { } @Test - def testResetOffsetsNewConsumerExistingTopic(): Unit = { + def testResetOffsetsExistingTopic(): Unit = { val args = Array("--bootstrap-server", brokerList, "--reset-offsets", "--group", "new.group", "--topic", topic, "--to-offset", "50") produceMessages(topic, 100) diff --git a/core/src/test/scala/unit/kafka/api/ApiUtilsTest.scala b/core/src/test/scala/unit/kafka/api/ApiUtilsTest.scala index b71b00bc05a5c..9fe4cbf1965b4 100644 --- a/core/src/test/scala/unit/kafka/api/ApiUtilsTest.scala +++ b/core/src/test/scala/unit/kafka/api/ApiUtilsTest.scala @@ -20,10 +20,12 @@ package kafka.api import org.junit._ import org.scalatest.junit.JUnitSuite import org.junit.Assert._ + import scala.util.Random import java.nio.ByteBuffer -import kafka.common.KafkaException + import kafka.utils.TestUtils +import org.apache.kafka.common.KafkaException object ApiUtilsTest { val rnd: Random = new Random() diff --git a/core/src/test/scala/unit/kafka/api/FetchRequestTest.scala b/core/src/test/scala/unit/kafka/api/FetchRequestTest.scala deleted file mode 100644 index c2bdf49517f7f..0000000000000 --- a/core/src/test/scala/unit/kafka/api/FetchRequestTest.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.api - -import kafka.common.TopicAndPartition -import org.junit.Assert.{assertEquals, assertNotEquals} -import org.junit.Test - -class FetchRequestTest { - - @Test - def testShuffle() { - val seq = (0 to 100).map { i => - val topic = s"topic${i % 10}" - (TopicAndPartition(topic, i / 10), PartitionFetchInfo(i, 50)) - } - val shuffled = FetchRequest.shuffle(seq) - assertEquals(seq.size, shuffled.size) - assertNotEquals(seq, shuffled) - - seq.foreach { case (tp1, fetchInfo1) => - shuffled.foreach { case (tp2, fetchInfo2) => - if (tp1 == tp2) - assertEquals(fetchInfo1, fetchInfo2) - } - } - - val topics = seq.map { case (TopicAndPartition(t, _), _) => t }.distinct - topics.foreach { topic => - val startIndex = shuffled.indexWhere { case (tp, _) => tp.topic == topic } - val endIndex = shuffled.lastIndexWhere { case (tp, _) => tp.topic == topic } - // all partitions for a given topic should appear in sequence - assertEquals(Set(topic), shuffled.slice(startIndex, endIndex + 1).map { case (tp, _) => tp.topic }.toSet) - } - - val shuffled2 = FetchRequest.shuffle(seq) - assertNotEquals(shuffled, shuffled2) - assertNotEquals(seq, shuffled2) - } - - @Test - def testShuffleWithSingleTopic() { - val seq = (0 to 50).map(i => (TopicAndPartition("topic", i), PartitionFetchInfo(i, 70))) - val shuffled = FetchRequest.shuffle(seq) - assertEquals(seq.size, shuffled.size) - assertNotEquals(seq, shuffled) - } - -} diff --git a/core/src/test/scala/unit/kafka/api/RequestResponseSerializationTest.scala b/core/src/test/scala/unit/kafka/api/RequestResponseSerializationTest.scala deleted file mode 100644 index 33f9352397c3a..0000000000000 --- a/core/src/test/scala/unit/kafka/api/RequestResponseSerializationTest.scala +++ /dev/null @@ -1,205 +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 kafka.common.{OffsetAndMetadata, OffsetMetadataAndError} -import kafka.common._ -import kafka.message.{ByteBufferMessageSet, Message} -import kafka.common.TopicAndPartition -import kafka.utils.TestUtils -import TestUtils.createBroker -import java.nio.ByteBuffer - -import org.apache.kafka.common.network.ListenerName -import org.apache.kafka.common.protocol.Errors -import org.apache.kafka.common.security.auth.SecurityProtocol -import org.apache.kafka.common.utils.Time -import org.junit._ -import org.scalatest.junit.JUnitSuite -import org.junit.Assert._ - - -object SerializationTestUtils { - private val topic1 = "test1" - private val topic2 = "test2" - private val partitionDataFetchResponse0 = new FetchResponsePartitionData(messages = new ByteBufferMessageSet(new Message("first message".getBytes))) - private val partitionDataFetchResponse1 = new FetchResponsePartitionData(messages = new ByteBufferMessageSet(new Message("second message".getBytes))) - private val partitionDataFetchResponse2 = new FetchResponsePartitionData(messages = new ByteBufferMessageSet(new Message("third message".getBytes))) - private val partitionDataFetchResponse3 = new FetchResponsePartitionData(messages = new ByteBufferMessageSet(new Message("fourth message".getBytes))) - private val partitionDataFetchResponseMap = Map((0, partitionDataFetchResponse0), (1, partitionDataFetchResponse1), (2, partitionDataFetchResponse2), (3, partitionDataFetchResponse3)) - - private val topicDataFetchResponse = { - val groupedData = Array(topic1, topic2).flatMap(topic => - partitionDataFetchResponseMap.map(partitionAndData => - (TopicAndPartition(topic, partitionAndData._1), partitionAndData._2))) - collection.immutable.Map(groupedData:_*) - } - - private val partitionDataMessage0 = new ByteBufferMessageSet(new Message("first message".getBytes)) - private val partitionDataMessage1 = new ByteBufferMessageSet(new Message("second message".getBytes)) - private val partitionDataMessage2 = new ByteBufferMessageSet(new Message("third message".getBytes)) - private val partitionDataMessage3 = new ByteBufferMessageSet(new Message("fourth message".getBytes)) - private val partitionDataProducerRequestArray = Array(partitionDataMessage0, partitionDataMessage1, partitionDataMessage2, partitionDataMessage3) - - val topicDataProducerRequest = { - val groupedData = Array(topic1, topic2).flatMap(topic => - partitionDataProducerRequestArray.zipWithIndex.map - { - case(partitionDataMessage, partition) => - (TopicAndPartition(topic, partition), partitionDataMessage) - }) - collection.mutable.Map(groupedData:_*) - } - - private val requestInfos = collection.immutable.Map( - TopicAndPartition(topic1, 0) -> PartitionFetchInfo(1000, 100), - TopicAndPartition(topic1, 1) -> PartitionFetchInfo(2000, 100), - TopicAndPartition(topic1, 2) -> PartitionFetchInfo(3000, 100), - TopicAndPartition(topic1, 3) -> PartitionFetchInfo(4000, 100), - TopicAndPartition(topic2, 0) -> PartitionFetchInfo(1000, 100), - TopicAndPartition(topic2, 1) -> PartitionFetchInfo(2000, 100), - TopicAndPartition(topic2, 2) -> PartitionFetchInfo(3000, 100), - TopicAndPartition(topic2, 3) -> PartitionFetchInfo(4000, 100) - ) - - private val brokers = List(createBroker(0, "localhost", 1011), createBroker(0, "localhost", 1012), - createBroker(0, "localhost", 1013)) - - def createTestFetchRequest: FetchRequest = new FetchRequest(requestInfo = requestInfos.toVector) - - def createTestFetchResponse: FetchResponse = FetchResponse(1, topicDataFetchResponse.toVector) - - def createTestOffsetRequest = new OffsetRequest( - collection.immutable.Map(TopicAndPartition(topic1, 1) -> PartitionOffsetRequestInfo(1000, 200)), - replicaId = 0 - ) - - def createTestOffsetResponse: OffsetResponse = { - new OffsetResponse(0, collection.immutable.Map( - TopicAndPartition(topic1, 1) -> PartitionOffsetsResponse(Errors.NONE, Seq(1000l, 2000l, 3000l, 4000l))) - ) - } - - def createTestOffsetCommitRequestV2: OffsetCommitRequest = { - new OffsetCommitRequest( - groupId = "group 1", - retentionMs = Time.SYSTEM.milliseconds, - requestInfo=collection.immutable.Map( - TopicAndPartition(topic1, 0) -> OffsetAndMetadata(42L, "some metadata"), - TopicAndPartition(topic1, 1) -> OffsetAndMetadata(100L, OffsetMetadata.NoMetadata) - )) - } - - def createTestOffsetCommitRequestV1: OffsetCommitRequest = { - new OffsetCommitRequest( - versionId = 1, - groupId = "group 1", - requestInfo = collection.immutable.Map( - TopicAndPartition(topic1, 0) -> OffsetAndMetadata(42L, "some metadata", Time.SYSTEM.milliseconds), - TopicAndPartition(topic1, 1) -> OffsetAndMetadata(100L, OffsetMetadata.NoMetadata, Time.SYSTEM.milliseconds) - )) - } - - def createTestOffsetCommitRequestV0: OffsetCommitRequest = { - new OffsetCommitRequest( - versionId = 0, - groupId = "group 1", - requestInfo = collection.immutable.Map( - TopicAndPartition(topic1, 0) -> OffsetAndMetadata(42L, "some metadata"), - TopicAndPartition(topic1, 1) -> OffsetAndMetadata(100L, OffsetMetadata.NoMetadata) - )) - } - - def createTestOffsetCommitResponse: OffsetCommitResponse = { - new OffsetCommitResponse(collection.immutable.Map(TopicAndPartition(topic1, 0) -> Errors.NONE, - TopicAndPartition(topic1, 1) -> Errors.NONE)) - } - - def createTestOffsetFetchRequest: OffsetFetchRequest = { - new OffsetFetchRequest("group 1", Seq( - TopicAndPartition(topic1, 0), - TopicAndPartition(topic1, 1) - )) - } - - def createTestOffsetFetchResponse: OffsetFetchResponse = { - new OffsetFetchResponse(collection.immutable.Map( - TopicAndPartition(topic1, 0) -> OffsetMetadataAndError(42L, "some metadata", Errors.NONE), - TopicAndPartition(topic1, 1) -> OffsetMetadataAndError(100L, OffsetMetadata.NoMetadata, Errors.UNKNOWN_TOPIC_OR_PARTITION) - ), error = Errors.NONE) - } - - def createConsumerMetadataRequest: GroupCoordinatorRequest = GroupCoordinatorRequest("group 1", clientId = "client 1") - - def createConsumerMetadataResponse: GroupCoordinatorResponse = { - GroupCoordinatorResponse(Some( - brokers.head.brokerEndPoint(ListenerName.forSecurityProtocol(SecurityProtocol.PLAINTEXT))), Errors.NONE, 0) - } -} - -class RequestResponseSerializationTest extends JUnitSuite { - private val fetchRequest = SerializationTestUtils.createTestFetchRequest - private val offsetRequest = SerializationTestUtils.createTestOffsetRequest - private val offsetResponse = SerializationTestUtils.createTestOffsetResponse - 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 = GroupCoordinatorResponse(None, Errors.COORDINATOR_NOT_AVAILABLE, 0) - - @Test - def testSerializationAndDeserialization() { - - val requestsAndResponses = - collection.immutable.Seq(fetchRequest, offsetRequest, offsetResponse, - offsetCommitRequestV0, offsetCommitRequestV1, offsetCommitRequestV2, offsetCommitResponse, - offsetFetchRequest, offsetFetchResponse, - consumerMetadataRequest, consumerMetadataResponse, - consumerMetadataResponseNoCoordinator) - - requestsAndResponses.foreach { original => - val buffer = ByteBuffer.allocate(original.sizeInBytes) - original.writeTo(buffer) - buffer.rewind() - val deserializer = original.getClass.getDeclaredMethod("readFrom", classOf[ByteBuffer]) - val deserialized = deserializer.invoke(null, buffer) - assertFalse("All serialized bytes in " + original.getClass.getSimpleName + " should have been consumed", - buffer.hasRemaining) - assertEquals("The original and deserialized for " + original.getClass.getSimpleName + " should be the same.", original, deserialized) - } - } - - @Test - def testFetchResponseVersion() { - val oldClientResponse = FetchResponse(1, Map( - TopicAndPartition("t1", 0) -> new FetchResponsePartitionData(messages = new ByteBufferMessageSet(new Message("first message".getBytes))) - ).toVector, 0) - - val newClientResponse = FetchResponse(1, Map( - TopicAndPartition("t1", 0) -> new FetchResponsePartitionData(messages = new ByteBufferMessageSet(new Message("first message".getBytes))) - ).toVector, 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/PartitionTest.scala b/core/src/test/scala/unit/kafka/cluster/PartitionTest.scala new file mode 100644 index 0000000000000..74c4f23fb3c34 --- /dev/null +++ b/core/src/test/scala/unit/kafka/cluster/PartitionTest.scala @@ -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 kafka.cluster + +import java.io.File +import java.nio.ByteBuffer +import java.util.Properties +import java.util.concurrent.CountDownLatch +import java.util.concurrent.atomic.AtomicBoolean + +import kafka.api.Request +import kafka.common.UnexpectedAppendOffsetException +import kafka.log.{CleanerConfig, LogConfig, LogManager} +import kafka.server._ +import kafka.utils.{MockScheduler, MockTime, TestUtils} +import kafka.zk.KafkaZkClient +import org.apache.kafka.common.TopicPartition +import org.apache.kafka.common.errors.ReplicaNotAvailableException +import org.apache.kafka.common.metrics.Metrics +import org.apache.kafka.common.utils.Utils +import org.apache.kafka.common.record._ +import org.apache.kafka.common.requests.{EpochEndOffset, LeaderAndIsrRequest} +import org.junit.{After, Before, Test} +import org.junit.Assert._ +import org.scalatest.Assertions.assertThrows +import org.easymock.EasyMock + +import scala.collection.JavaConverters._ + +class PartitionTest { + + val brokerId = 101 + val topicPartition = new TopicPartition("test-topic", 0) + val time = new MockTime() + val brokerTopicStats = new BrokerTopicStats + val metrics = new Metrics + + var tmpDir: File = _ + var logDir1: File = _ + var logDir2: File = _ + var replicaManager: ReplicaManager = _ + var logManager: LogManager = _ + var logConfig: LogConfig = _ + + @Before + def setup(): Unit = { + val logProps = createLogProperties(Map.empty) + logConfig = LogConfig(logProps) + + tmpDir = TestUtils.tempDir() + logDir1 = TestUtils.randomPartitionLogDir(tmpDir) + logDir2 = TestUtils.randomPartitionLogDir(tmpDir) + logManager = TestUtils.createLogManager( + logDirs = Seq(logDir1, logDir2), defaultConfig = logConfig, CleanerConfig(enableCleaner = false), time) + logManager.startup() + + val brokerProps = TestUtils.createBrokerConfig(brokerId, TestUtils.MockZkConnect) + brokerProps.put(KafkaConfig.LogDirsProp, Seq(logDir1, logDir2).map(_.getAbsolutePath).mkString(",")) + val brokerConfig = KafkaConfig.fromProps(brokerProps) + val kafkaZkClient = EasyMock.createMock(classOf[KafkaZkClient]) + replicaManager = new ReplicaManager( + config = brokerConfig, metrics, time, zkClient = kafkaZkClient, new MockScheduler(time), + logManager, new AtomicBoolean(false), QuotaFactory.instantiate(brokerConfig, metrics, time, ""), + brokerTopicStats, new MetadataCache(brokerId), new LogDirFailureChannel(brokerConfig.logDirs.size)) + + EasyMock.expect(kafkaZkClient.getEntityConfigs(EasyMock.anyString(), EasyMock.anyString())).andReturn(logProps).anyTimes() + EasyMock.expect(kafkaZkClient.conditionalUpdatePath(EasyMock.anyObject(), EasyMock.anyObject(), EasyMock.anyObject(), EasyMock.anyObject())) + .andReturn((true, 0)).anyTimes() + EasyMock.replay(kafkaZkClient) + } + + private def createLogProperties(overrides: Map[String, String]): Properties = { + val logProps = new Properties() + logProps.put(LogConfig.SegmentBytesProp, 512: java.lang.Integer) + logProps.put(LogConfig.SegmentIndexBytesProp, 1000: java.lang.Integer) + logProps.put(LogConfig.RetentionMsProp, 999: java.lang.Integer) + overrides.foreach { case (k, v) => logProps.put(k, v) } + logProps + } + + @After + def tearDown(): Unit = { + brokerTopicStats.close() + metrics.close() + + logManager.shutdown() + Utils.delete(tmpDir) + logManager.liveLogDirs.foreach(Utils.delete) + replicaManager.shutdown(checkpointHW = false) + } + + @Test + def testMakeLeaderUpdatesEpochCache(): Unit = { + val controllerEpoch = 3 + val leader = brokerId + val follower = brokerId + 1 + val controllerId = brokerId + 3 + val replicas = List[Integer](leader, follower).asJava + val isr = List[Integer](leader, follower).asJava + val leaderEpoch = 8 + + val log = logManager.getOrCreateLog(topicPartition, logConfig) + log.appendAsLeader(MemoryRecords.withRecords(0L, CompressionType.NONE, 0, + new SimpleRecord("k1".getBytes, "v1".getBytes), + new SimpleRecord("k2".getBytes, "v2".getBytes) + ), leaderEpoch = 0) + log.appendAsLeader(MemoryRecords.withRecords(0L, CompressionType.NONE, 5, + new SimpleRecord("k3".getBytes, "v3".getBytes), + new SimpleRecord("k4".getBytes, "v4".getBytes) + ), leaderEpoch = 5) + assertEquals(4, log.logEndOffset) + + val partition = new Partition(topicPartition.topic, topicPartition.partition, time, replicaManager) + assertTrue("Expected makeLeader to succeed", + partition.makeLeader(controllerId, new LeaderAndIsrRequest.PartitionState(controllerEpoch, leader, leaderEpoch, + isr, 1, replicas, true), 0)) + + assertEquals(Some(4), partition.leaderReplicaIfLocal.map(_.logEndOffset.messageOffset)) + + val epochEndOffset = partition.lastOffsetForLeaderEpoch(leaderEpoch) + assertEquals(4, epochEndOffset.endOffset) + assertEquals(leaderEpoch, epochEndOffset.leaderEpoch) + } + + @Test + def testMakeLeaderDoesNotUpdateEpochCacheForOldFormats(): Unit = { + val controllerEpoch = 3 + val leader = brokerId + val follower = brokerId + 1 + val controllerId = brokerId + 3 + val replicas = List[Integer](leader, follower).asJava + val isr = List[Integer](leader, follower).asJava + + val leaderEpoch = 8 + + val logConfig = LogConfig(createLogProperties(Map( + LogConfig.MessageFormatVersionProp -> kafka.api.KAFKA_0_10_2_IV0.shortVersion))) + val log = logManager.getOrCreateLog(topicPartition, logConfig) + log.appendAsLeader(TestUtils.records(List( + new SimpleRecord("k1".getBytes, "v1".getBytes), + new SimpleRecord("k2".getBytes, "v2".getBytes)), + magicValue = RecordVersion.V1.value + ), leaderEpoch = 0) + log.appendAsLeader(TestUtils.records(List( + new SimpleRecord("k3".getBytes, "v3".getBytes), + new SimpleRecord("k4".getBytes, "v4".getBytes)), + magicValue = RecordVersion.V1.value + ), leaderEpoch = 5) + assertEquals(4, log.logEndOffset) + + val partition = new Partition(topicPartition.topic, topicPartition.partition, time, replicaManager) + assertTrue("Expected makeLeader to succeed", + partition.makeLeader(controllerId, new LeaderAndIsrRequest.PartitionState(controllerEpoch, leader, leaderEpoch, + isr, 1, replicas, true), 0)) + + assertEquals(Some(4), partition.leaderReplicaIfLocal.map(_.logEndOffset.messageOffset)) + assertEquals(EpochEndOffset.UNDEFINED_EPOCH, log.leaderEpochCache.latestEpoch) + + val epochEndOffset = partition.lastOffsetForLeaderEpoch(leaderEpoch) + assertEquals(EpochEndOffset.UNDEFINED_EPOCH_OFFSET, epochEndOffset.endOffset) + assertEquals(EpochEndOffset.UNDEFINED_EPOCH, epochEndOffset.leaderEpoch) + } + + @Test + // Verify that partition.removeFutureLocalReplica() and partition.maybeReplaceCurrentWithFutureReplica() can run concurrently + def testMaybeReplaceCurrentWithFutureReplica(): Unit = { + val latch = new CountDownLatch(1) + + logManager.maybeUpdatePreferredLogDir(topicPartition, logDir1.getAbsolutePath) + val log1 = logManager.getOrCreateLog(topicPartition, logConfig) + logManager.maybeUpdatePreferredLogDir(topicPartition, logDir2.getAbsolutePath) + val log2 = logManager.getOrCreateLog(topicPartition, logConfig, isFuture = true) + val currentReplica = new Replica(brokerId, topicPartition, time, log = Some(log1)) + val futureReplica = new Replica(Request.FutureLocalReplicaId, topicPartition, time, log = Some(log2)) + val partition = new Partition(topicPartition.topic, topicPartition.partition, time, replicaManager) + + partition.addReplicaIfNotExists(futureReplica) + partition.addReplicaIfNotExists(currentReplica) + assertEquals(Some(currentReplica), partition.getReplica(brokerId)) + assertEquals(Some(futureReplica), partition.getReplica(Request.FutureLocalReplicaId)) + + val thread1 = new Thread { + override def run(): Unit = { + latch.await() + partition.removeFutureLocalReplica() + } + } + + val thread2 = new Thread { + override def run(): Unit = { + latch.await() + partition.maybeReplaceCurrentWithFutureReplica() + } + } + + thread1.start() + thread2.start() + + latch.countDown() + thread1.join() + thread2.join() + assertEquals(None, partition.getReplica(Request.FutureLocalReplicaId)) + } + + // Verify that replacement works when the replicas have the same log end offset but different base offsets in the + // active segment + @Test + def testMaybeReplaceCurrentWithFutureReplicaDifferentBaseOffsets(): Unit = { + // Write records with duplicate keys to current replica and roll at offset 6 + logManager.maybeUpdatePreferredLogDir(topicPartition, logDir1.getAbsolutePath) + val log1 = logManager.getOrCreateLog(topicPartition, logConfig) + log1.appendAsLeader(MemoryRecords.withRecords(0L, CompressionType.NONE, 0, + new SimpleRecord("k1".getBytes, "v1".getBytes), + new SimpleRecord("k1".getBytes, "v2".getBytes), + new SimpleRecord("k1".getBytes, "v3".getBytes), + new SimpleRecord("k2".getBytes, "v4".getBytes), + new SimpleRecord("k2".getBytes, "v5".getBytes), + new SimpleRecord("k2".getBytes, "v6".getBytes) + ), leaderEpoch = 0) + log1.roll() + log1.appendAsLeader(MemoryRecords.withRecords(0L, CompressionType.NONE, 0, + new SimpleRecord("k3".getBytes, "v7".getBytes), + new SimpleRecord("k4".getBytes, "v8".getBytes) + ), leaderEpoch = 0) + + // Write to the future replica as if the log had been compacted, and do not roll the segment + logManager.maybeUpdatePreferredLogDir(topicPartition, logDir2.getAbsolutePath) + val log2 = logManager.getOrCreateLog(topicPartition, logConfig, isFuture = true) + val buffer = ByteBuffer.allocate(1024) + var builder = MemoryRecords.builder(buffer, RecordBatch.CURRENT_MAGIC_VALUE, CompressionType.NONE, + TimestampType.CREATE_TIME, 0L, RecordBatch.NO_TIMESTAMP, 0) + builder.appendWithOffset(2L, new SimpleRecord("k1".getBytes, "v3".getBytes)) + builder.appendWithOffset(5L, new SimpleRecord("k2".getBytes, "v6".getBytes)) + builder.appendWithOffset(6L, new SimpleRecord("k3".getBytes, "v7".getBytes)) + builder.appendWithOffset(7L, new SimpleRecord("k4".getBytes, "v8".getBytes)) + + log2.appendAsFollower(builder.build()) + + val currentReplica = new Replica(brokerId, topicPartition, time, log = Some(log1)) + val futureReplica = new Replica(Request.FutureLocalReplicaId, topicPartition, time, log = Some(log2)) + val partition = new Partition(topicPartition.topic, topicPartition.partition, time, replicaManager) + + partition.addReplicaIfNotExists(futureReplica) + partition.addReplicaIfNotExists(currentReplica) + assertEquals(Some(currentReplica), partition.getReplica(brokerId)) + assertEquals(Some(futureReplica), partition.getReplica(Request.FutureLocalReplicaId)) + + assertTrue(partition.maybeReplaceCurrentWithFutureReplica()) + } + + @Test + def testAppendRecordsAsFollowerBelowLogStartOffset(): Unit = { + val log = logManager.getOrCreateLog(topicPartition, logConfig) + val replica = new Replica(brokerId, topicPartition, time, log = Some(log)) + val partition = new Partition(topicPartition.topic, topicPartition.partition, time, replicaManager) + partition.addReplicaIfNotExists(replica) + assertEquals(Some(replica), partition.getReplica(replica.brokerId)) + + val initialLogStartOffset = 5L + partition.truncateFullyAndStartAt(initialLogStartOffset, isFuture = false) + assertEquals(s"Log end offset after truncate fully and start at $initialLogStartOffset:", + initialLogStartOffset, replica.logEndOffset.messageOffset) + assertEquals(s"Log start offset after truncate fully and start at $initialLogStartOffset:", + initialLogStartOffset, replica.logStartOffset) + + // verify that we cannot append records that do not contain log start offset even if the log is empty + assertThrows[UnexpectedAppendOffsetException] { + // append one record with offset = 3 + partition.appendRecordsToFollowerOrFutureReplica(createRecords(List(new SimpleRecord("k1".getBytes, "v1".getBytes)), baseOffset = 3L), isFuture = false) + } + assertEquals(s"Log end offset should not change after failure to append", initialLogStartOffset, replica.logEndOffset.messageOffset) + + // verify that we can append records that contain log start offset, even when first + // offset < log start offset if the log is empty + val newLogStartOffset = 4L + val records = createRecords(List(new SimpleRecord("k1".getBytes, "v1".getBytes), + new SimpleRecord("k2".getBytes, "v2".getBytes), + new SimpleRecord("k3".getBytes, "v3".getBytes)), + baseOffset = newLogStartOffset) + partition.appendRecordsToFollowerOrFutureReplica(records, isFuture = false) + assertEquals(s"Log end offset after append of 3 records with base offset $newLogStartOffset:", 7L, replica.logEndOffset.messageOffset) + assertEquals(s"Log start offset after append of 3 records with base offset $newLogStartOffset:", newLogStartOffset, replica.logStartOffset) + + // and we can append more records after that + partition.appendRecordsToFollowerOrFutureReplica(createRecords(List(new SimpleRecord("k1".getBytes, "v1".getBytes)), baseOffset = 7L), isFuture = false) + assertEquals(s"Log end offset after append of 1 record at offset 7:", 8L, replica.logEndOffset.messageOffset) + assertEquals(s"Log start offset not expected to change:", newLogStartOffset, replica.logStartOffset) + + // but we cannot append to offset < log start if the log is not empty + assertThrows[UnexpectedAppendOffsetException] { + val records2 = createRecords(List(new SimpleRecord("k1".getBytes, "v1".getBytes), + new SimpleRecord("k2".getBytes, "v2".getBytes)), + baseOffset = 3L) + partition.appendRecordsToFollowerOrFutureReplica(records2, isFuture = false) + } + assertEquals(s"Log end offset should not change after failure to append", 8L, replica.logEndOffset.messageOffset) + + // we still can append to next offset + partition.appendRecordsToFollowerOrFutureReplica(createRecords(List(new SimpleRecord("k1".getBytes, "v1".getBytes)), baseOffset = 8L), isFuture = false) + assertEquals(s"Log end offset after append of 1 record at offset 8:", 9L, replica.logEndOffset.messageOffset) + assertEquals(s"Log start offset not expected to change:", newLogStartOffset, replica.logStartOffset) + } + + @Test + def testGetReplica(): Unit = { + val log = logManager.getOrCreateLog(topicPartition, logConfig) + val replica = new Replica(brokerId, topicPartition, time, log = Some(log)) + val partition = new + Partition(topicPartition.topic, topicPartition.partition, time, replicaManager) + + assertEquals(None, partition.getReplica(brokerId)) + assertThrows[ReplicaNotAvailableException] { + partition.getReplicaOrException(brokerId) + } + + partition.addReplicaIfNotExists(replica) + assertEquals(replica, partition.getReplicaOrException(brokerId)) + } + + @Test + def testAppendRecordsToFollowerWithNoReplicaThrowsException(): Unit = { + val partition = new Partition(topicPartition.topic, topicPartition.partition, time, replicaManager) + assertThrows[ReplicaNotAvailableException] { + partition.appendRecordsToFollowerOrFutureReplica( + createRecords(List(new SimpleRecord("k1".getBytes, "v1".getBytes)), baseOffset = 0L), isFuture = false) + } + } + + @Test + def testMakeFollowerWithNoLeaderIdChange(): Unit = { + val partition = new Partition(topicPartition.topic, topicPartition.partition, time, replicaManager) + + // Start off as follower + var partitionStateInfo = new LeaderAndIsrRequest.PartitionState(0, 1, 1, List[Integer](0, 1, 2).asJava, 1, List[Integer](0, 1, 2).asJava, false) + partition.makeFollower(0, partitionStateInfo, 0) + + // Request with same leader and epoch increases by more than 1, perform become-follower steps + partitionStateInfo = new LeaderAndIsrRequest.PartitionState(0, 1, 3, List[Integer](0, 1, 2).asJava, 1, List[Integer](0, 1, 2).asJava, false) + assertTrue(partition.makeFollower(0, partitionStateInfo, 1)) + + // Request with same leader and epoch increases by only 1, skip become-follower steps + partitionStateInfo = new LeaderAndIsrRequest.PartitionState(0, 1, 4, List[Integer](0, 1, 2).asJava, 1, List[Integer](0, 1, 2).asJava, false) + assertFalse(partition.makeFollower(0, partitionStateInfo, 2)) + + // Request with same leader and same epoch, skip become-follower steps + partitionStateInfo = new LeaderAndIsrRequest.PartitionState(0, 1, 4, List[Integer](0, 1, 2).asJava, 1, List[Integer](0, 1, 2).asJava, false) + assertFalse(partition.makeFollower(0, partitionStateInfo, 2)) + } + + @Test + def testFollowerDoesNotJoinISRUntilCaughtUpToOffsetWithinCurrentLeaderEpoch(): Unit = { + val controllerEpoch = 3 + val leader = brokerId + val follower1 = brokerId + 1 + val follower2 = brokerId + 2 + val controllerId = brokerId + 3 + val replicas = List[Integer](leader, follower1, follower2).asJava + val isr = List[Integer](leader, follower2).asJava + val leaderEpoch = 8 + val batch1 = TestUtils.records(records = List(new SimpleRecord("k1".getBytes, "v1".getBytes), + new SimpleRecord("k2".getBytes, "v2".getBytes))) + val batch2 = TestUtils.records(records = List(new SimpleRecord("k3".getBytes, "v1".getBytes), + new SimpleRecord("k4".getBytes, "v2".getBytes), + new SimpleRecord("k5".getBytes, "v3".getBytes))) + val batch3 = TestUtils.records(records = List(new SimpleRecord("k6".getBytes, "v1".getBytes), + new SimpleRecord("k7".getBytes, "v2".getBytes))) + + val partition = new Partition(topicPartition.topic, topicPartition.partition, time, replicaManager) + assertTrue("Expected first makeLeader() to return 'leader changed'", + partition.makeLeader(controllerId, new LeaderAndIsrRequest.PartitionState(controllerEpoch, leader, leaderEpoch, isr, 1, replicas, true), 0)) + assertEquals("Current leader epoch", leaderEpoch, partition.getLeaderEpoch) + assertEquals("ISR", Set[Integer](leader, follower2), partition.inSyncReplicas.map(_.brokerId)) + + // after makeLeader(() call, partition should know about all the replicas + val leaderReplica = partition.getReplica(leader).get + val follower1Replica = partition.getReplica(follower1).get + val follower2Replica = partition.getReplica(follower2).get + + // append records with initial leader epoch + val lastOffsetOfFirstBatch = partition.appendRecordsToLeader(batch1, isFromClient = true).lastOffset + partition.appendRecordsToLeader(batch2, isFromClient = true) + assertEquals("Expected leader's HW not move", leaderReplica.logStartOffset, leaderReplica.highWatermark.messageOffset) + + // let the follower in ISR move leader's HW to move further but below LEO + def readResult(fetchInfo: FetchDataInfo, leaderReplica: Replica): LogReadResult = { + LogReadResult(info = fetchInfo, + highWatermark = leaderReplica.highWatermark.messageOffset, + leaderLogStartOffset = leaderReplica.logStartOffset, + leaderLogEndOffset = leaderReplica.logEndOffset.messageOffset, + followerLogStartOffset = 0, + fetchTimeMs = time.milliseconds, + readSize = 10240, + lastStableOffset = None) + } + partition.updateReplicaLogReadResult( + follower2Replica, readResult(FetchDataInfo(LogOffsetMetadata(0), batch1), leaderReplica)) + partition.updateReplicaLogReadResult( + follower2Replica, readResult(FetchDataInfo(LogOffsetMetadata(lastOffsetOfFirstBatch), batch2), leaderReplica)) + assertEquals("Expected leader's HW", lastOffsetOfFirstBatch, leaderReplica.highWatermark.messageOffset) + + // current leader becomes follower and then leader again (without any new records appended) + partition.makeFollower( + controllerId, new LeaderAndIsrRequest.PartitionState(controllerEpoch, follower2, leaderEpoch + 1, isr, 1, replicas, false), 1) + assertTrue("Expected makeLeader() to return 'leader changed' after makeFollower()", + partition.makeLeader(controllerEpoch, new LeaderAndIsrRequest.PartitionState( + controllerEpoch, leader, leaderEpoch + 2, isr, 1, replicas, false), 2)) + val currentLeaderEpochStartOffset = leaderReplica.logEndOffset.messageOffset + + // append records with the latest leader epoch + partition.appendRecordsToLeader(batch3, isFromClient = true) + + // fetch from follower not in ISR from log start offset should not add this follower to ISR + partition.updateReplicaLogReadResult(follower1Replica, + readResult(FetchDataInfo(LogOffsetMetadata(0), batch1), leaderReplica)) + partition.updateReplicaLogReadResult(follower1Replica, + readResult(FetchDataInfo(LogOffsetMetadata(lastOffsetOfFirstBatch), batch2), leaderReplica)) + assertEquals("ISR", Set[Integer](leader, follower2), partition.inSyncReplicas.map(_.brokerId)) + + // fetch from the follower not in ISR from start offset of the current leader epoch should + // add this follower to ISR + partition.updateReplicaLogReadResult(follower1Replica, + readResult(FetchDataInfo(LogOffsetMetadata(currentLeaderEpochStartOffset), batch3), leaderReplica)) + assertEquals("ISR", Set[Integer](leader, follower1, follower2), partition.inSyncReplicas.map(_.brokerId)) + } + + def createRecords(records: Iterable[SimpleRecord], baseOffset: Long, partitionLeaderEpoch: Int = 0): MemoryRecords = { + val buf = ByteBuffer.allocate(DefaultRecordBatch.sizeInBytes(records.asJava)) + val builder = MemoryRecords.builder( + buf, RecordBatch.CURRENT_MAGIC_VALUE, CompressionType.NONE, TimestampType.LOG_APPEND_TIME, + baseOffset, time.milliseconds, partitionLeaderEpoch) + records.foreach(builder.append) + builder.build() + } + +} diff --git a/core/src/test/scala/unit/kafka/common/ConfigTest.scala b/core/src/test/scala/unit/kafka/common/ConfigTest.scala deleted file mode 100644 index a506d52218d9a..0000000000000 --- a/core/src/test/scala/unit/kafka/common/ConfigTest.scala +++ /dev/null @@ -1,57 +0,0 @@ -/** - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package kafka.common - -import org.junit.Assert._ -import collection.mutable.ArrayBuffer -import org.junit.Test -import kafka.consumer.ConsumerConfig - -class ConfigTest { - - @Test - def testInvalidGroupIds() { - val invalidGroupIds = new ArrayBuffer[String]() - val badChars = Array('/', '\\', ',', '\u0000', ':', "\"", '\'', ';', '*', '?', ' ', '\t', '\r', '\n', '=') - for (weirdChar <- badChars) { - invalidGroupIds += "Is" + weirdChar + "illegal" - } - - for (i <- 0 until invalidGroupIds.size) { - try { - ConsumerConfig.validateGroupId(invalidGroupIds(i)) - fail("Should throw InvalidGroupIdException.") - } - catch { - case _: InvalidConfigException => // This is good - } - } - - val validGroupIds = new ArrayBuffer[String]() - validGroupIds += ("valid", "GROUP", "iDs", "ar6", "VaL1d", "_0-9_.", "") - for (i <- 0 until validGroupIds.size) { - try { - ConsumerConfig.validateGroupId(validGroupIds(i)) - } - catch { - case _: Exception => fail("Should not throw exception.") - } - } - } -} - diff --git a/core/src/test/scala/unit/kafka/common/ZkNodeChangeNotificationListenerTest.scala b/core/src/test/scala/unit/kafka/common/ZkNodeChangeNotificationListenerTest.scala index cee0bd6b72582..0462300b248ac 100644 --- a/core/src/test/scala/unit/kafka/common/ZkNodeChangeNotificationListenerTest.scala +++ b/core/src/test/scala/unit/kafka/common/ZkNodeChangeNotificationListenerTest.scala @@ -16,44 +16,46 @@ */ package kafka.common -import kafka.security.auth.{Group, Literal, Resource} +import kafka.security.auth.{Group, Resource} import kafka.utils.TestUtils -import kafka.zk.{AclChangeNotificationSequenceZNode, ZkAclStore, ZooKeeperTestHarness} -import org.junit.{After, Test} +import kafka.zk.{LiteralAclChangeStore, LiteralAclStore, ZkAclChangeStore, ZooKeeperTestHarness} +import org.apache.kafka.common.resource.PatternType.LITERAL +import org.junit.{After, Before, Test} + +import scala.collection.mutable.ArrayBuffer class ZkNodeChangeNotificationListenerTest extends ZooKeeperTestHarness { - var notificationListener: ZkNodeChangeNotificationListener = _ + private val changeExpirationMs = 1000 + private var notificationListener: ZkNodeChangeNotificationListener = _ + private var notificationHandler: TestNotificationHandler = _ + + @Before + override def setUp(): Unit = { + super.setUp() + zkClient.createAclPaths() + notificationHandler = new TestNotificationHandler() + } @After override def tearDown(): Unit = { if (notificationListener != null) { notificationListener.close() } + super.tearDown() } @Test def testProcessNotification() { - @volatile var notification: Resource = null - @volatile var invocationCount = 0 - val notificationHandler = new NotificationHandler { - override def processNotification(notificationMessage: Array[Byte]): Unit = { - notification = AclChangeNotificationSequenceZNode.decode(Literal, notificationMessage) - invocationCount += 1 - } - } + val notificationMessage1 = Resource(Group, "messageA", LITERAL) + val notificationMessage2 = Resource(Group, "messageB", LITERAL) - zkClient.createAclPaths() - val notificationMessage1 = Resource(Group, "messageA", Literal) - val notificationMessage2 = Resource(Group, "messageB", Literal) - val changeExpirationMs = 1000 - - notificationListener = new ZkNodeChangeNotificationListener(zkClient, ZkAclStore(Literal).aclChangePath, - AclChangeNotificationSequenceZNode.SequenceNumberPrefix, notificationHandler, changeExpirationMs) + notificationListener = new ZkNodeChangeNotificationListener(zkClient, LiteralAclChangeStore.aclChangePath, + ZkAclChangeStore.SequenceNumberPrefix, notificationHandler, changeExpirationMs) notificationListener.init() zkClient.createAclChangeNotification(notificationMessage1) - TestUtils.waitUntilTrue(() => invocationCount == 1 && notification == notificationMessage1, + TestUtils.waitUntilTrue(() => notificationHandler.received().size == 1 && notificationHandler.received().last == notificationMessage1, "Failed to send/process notification message in the timeout period.") /* @@ -65,12 +67,43 @@ class ZkNodeChangeNotificationListenerTest extends ZooKeeperTestHarness { */ zkClient.createAclChangeNotification(notificationMessage2) - TestUtils.waitUntilTrue(() => invocationCount == 2 && notification == notificationMessage2, + TestUtils.waitUntilTrue(() => notificationHandler.received().size == 2 && notificationHandler.received().last == notificationMessage2, "Failed to send/process notification message in the timeout period.") - (3 to 10).foreach(i => zkClient.createAclChangeNotification(Resource(Group, "message" + i, Literal))) + (3 to 10).foreach(i => zkClient.createAclChangeNotification(Resource(Group, "message" + i, LITERAL))) + + TestUtils.waitUntilTrue(() => notificationHandler.received().size == 10, + s"Expected 10 invocations of processNotifications, but there were ${notificationHandler.received()}") + } + + @Test + def testSwallowsProcessorException() : Unit = { + notificationHandler.setThrowSize(2) + notificationListener = new ZkNodeChangeNotificationListener(zkClient, LiteralAclChangeStore.aclChangePath, + ZkAclChangeStore.SequenceNumberPrefix, notificationHandler, changeExpirationMs) + notificationListener.init() + + zkClient.createAclChangeNotification(Resource(Group, "messageA", LITERAL)) + zkClient.createAclChangeNotification(Resource(Group, "messageB", LITERAL)) + zkClient.createAclChangeNotification(Resource(Group, "messageC", LITERAL)) + + TestUtils.waitUntilTrue(() => notificationHandler.received().size == 3, + s"Expected 2 invocations of processNotifications, but there were ${notificationHandler.received()}") + } + + private class TestNotificationHandler extends NotificationHandler { + private val messages = ArrayBuffer.empty[Resource] + @volatile private var throwSize = Option.empty[Int] + + override def processNotification(notificationMessage: Array[Byte]): Unit = { + messages += LiteralAclStore.changeStore.decode(notificationMessage) + + if (throwSize.contains(messages.size)) + throw new RuntimeException("Oh no, my processing failed!") + } + + def received(): Seq[Resource] = messages - TestUtils.waitUntilTrue(() => invocationCount == 10 , - s"Expected 10 invocations of processNotifications, but there were $invocationCount") + def setThrowSize(index: Int): Unit = throwSize = Option(index) } -} +} \ No newline at end of file diff --git a/core/src/test/scala/unit/kafka/consumer/ConsumerIteratorTest.scala b/core/src/test/scala/unit/kafka/consumer/ConsumerIteratorTest.scala deleted file mode 100755 index c495764558aee..0000000000000 --- a/core/src/test/scala/unit/kafka/consumer/ConsumerIteratorTest.scala +++ /dev/null @@ -1,121 +0,0 @@ - -/** - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package kafka.consumer - -import java.util.concurrent._ -import java.util.concurrent.atomic._ - -import kafka.common.LongRef - -import scala.collection._ -import org.junit.Assert._ -import kafka.message._ -import kafka.server._ -import kafka.utils._ -import org.junit.{Before, Test} -import kafka.serializer._ -import kafka.integration.KafkaServerTestHarness - -@deprecated("This test has been deprecated and will be removed in a future release.", "0.11.0.0") -class ConsumerIteratorTest extends KafkaServerTestHarness { - - val numNodes = 1 - - 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 queue = new LinkedBlockingQueue[FetchedDataChunk] - var topicInfos: Seq[PartitionTopicInfo] = null - - def consumerConfig = new ConsumerConfig(TestUtils.createConsumerProperties(zkConnect, group, consumer0)) - - @Before - override def setUp() { - super.setUp() - topicInfos = configs.map(_ => new PartitionTopicInfo(topic, - 0, - queue, - new AtomicLong(consumedOffset), - new AtomicLong(0), - new AtomicInteger(0), - "")) - createTopic(topic, partitionReplicaAssignment = Map(0 -> Seq(configs.head.brokerId))) - } - - @Test - def testConsumerIteratorDeduplicationDeepIterator() { - val messageStrings = (0 until 10).map(_.toString).toList - val messages = messageStrings.map(s => new Message(s.getBytes)) - val messageSet = new ByteBufferMessageSet(DefaultCompressionCodec, new LongRef(0), messages:_*) - - topicInfos.head.enqueue(messageSet) - assertEquals(1, queue.size) - queue.put(ZookeeperConsumerConnector.shutdownCommand) - - val iter = new ConsumerIterator[String, String](queue, - consumerConfig.consumerTimeoutMs, - new StringDecoder(), - new StringDecoder(), - clientId = "") - val receivedMessages = (0 until 5).map(_ => iter.next.message) - - assertFalse(iter.hasNext) - assertEquals(0, queue.size) // Shutdown command has been consumed. - assertEquals(5, receivedMessages.size) - val unconsumed = messageSet.filter(_.offset >= consumedOffset).map(m => TestUtils.readString(m.message.payload)) - assertEquals(unconsumed, receivedMessages) - } - - @Test - def testConsumerIteratorDecodingFailure() { - val messageStrings = (0 until 10).map(_.toString).toList - val messages = messageStrings.map(s => new Message(s.getBytes)) - val messageSet = new ByteBufferMessageSet(NoCompressionCodec, new LongRef(0), messages:_*) - - topicInfos.head.enqueue(messageSet) - assertEquals(1, queue.size) - - val iter = new ConsumerIterator[String, String](queue, - ConsumerConfig.ConsumerTimeoutMs, - new FailDecoder(), - new FailDecoder(), - clientId = "") - - (0 until 5).foreach { i => - assertTrue(iter.hasNext) - val message = iter.next - assertEquals(message.offset, i + consumedOffset) - - try message.message // should fail - catch { - case _: UnsupportedOperationException => // this is ok - } - } - } - - class FailDecoder(props: VerifiableProperties = null) extends Decoder[String] { - def fromBytes(bytes: Array[Byte]): String = { - throw new UnsupportedOperationException("This decoder does not work at all..") - } - } -} diff --git a/core/src/test/scala/unit/kafka/consumer/PartitionAssignorTest.scala b/core/src/test/scala/unit/kafka/consumer/PartitionAssignorTest.scala deleted file mode 100644 index 12fcba6dc743b..0000000000000 --- a/core/src/test/scala/unit/kafka/consumer/PartitionAssignorTest.scala +++ /dev/null @@ -1,266 +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.consumer - -import org.easymock.EasyMock -import org.I0Itec.zkclient.ZkClient -import org.apache.zookeeper.data.Stat -import kafka.utils.{TestUtils, Logging, ZkUtils, Json} -import org.junit.Assert._ -import kafka.common.TopicAndPartition -import kafka.consumer.PartitionAssignorTest.StaticSubscriptionInfo -import kafka.consumer.PartitionAssignorTest.Scenario -import kafka.consumer.PartitionAssignorTest.WildcardSubscriptionInfo -import org.junit.Test - -@deprecated("This test has been deprecated and will be removed in a future release.", "0.11.0.0") -class PartitionAssignorTest extends Logging { - - @Test - def testRoundRobinPartitionAssignor() { - val assignor = new RoundRobinAssignor - - /** various scenarios with only wildcard consumers */ - (1 to PartitionAssignorTest.TestCaseCount).foreach { _ => - val consumerCount = 1.max(TestUtils.random.nextInt(PartitionAssignorTest.MaxConsumerCount + 1)) - val topicCount = PartitionAssignorTest.MinTopicCount.max(TestUtils.random.nextInt(PartitionAssignorTest.MaxTopicCount + 1)) - - val topicPartitionCounts = Map((1 to topicCount).map(topic => { - ("topic-" + topic, PartitionAssignorTest.MinPartitionCount.max(TestUtils.random.nextInt(PartitionAssignorTest.MaxPartitionCount))) - }):_*) - - val subscriptions = Map((1 to consumerCount).map { consumer => - val streamCount = 1.max(TestUtils.random.nextInt(PartitionAssignorTest.MaxStreamCount + 1)) - ("g1c" + consumer, WildcardSubscriptionInfo(streamCount, ".*", isWhitelist = true)) - }:_*) - val scenario = Scenario("g1", topicPartitionCounts, subscriptions) - val zkUtils = PartitionAssignorTest.setupZkClientMock(scenario) - EasyMock.replay(zkUtils.zkClient) - PartitionAssignorTest.assignAndVerify(scenario, assignor, zkUtils, verifyAssignmentIsUniform = true) - } - } - - @Test - def testRoundRobinPartitionAssignorStaticSubscriptions() { - val assignor = new RoundRobinAssignor - - /** test static subscription scenarios */ - (1 to PartitionAssignorTest.TestCaseCount).foreach (testCase => { - val consumerCount = 1.max(TestUtils.random.nextInt(PartitionAssignorTest.MaxConsumerCount + 1)) - val topicCount = PartitionAssignorTest.MinTopicCount.max(TestUtils.random.nextInt(PartitionAssignorTest.MaxTopicCount + 1)) - - val topicPartitionCounts = Map((1 to topicCount).map(topic => { - ("topic-" + topic, PartitionAssignorTest.MinPartitionCount.max(TestUtils.random.nextInt(PartitionAssignorTest.MaxPartitionCount))) - }).toSeq:_*) - - val subscriptions = Map((1 to consumerCount).map(consumer => { - val streamCounts = Map((1 to topicCount).map(topic => { - ("topic-" + topic, 1) - }).toSeq:_*) - ("g1c" + consumer, StaticSubscriptionInfo(streamCounts)) - }).toSeq:_*) - val scenario = Scenario("g1", topicPartitionCounts, subscriptions) - val zkUtils = PartitionAssignorTest.setupZkClientMock(scenario) - EasyMock.replay(zkUtils.zkClient) - PartitionAssignorTest.assignAndVerify(scenario, assignor, zkUtils, verifyAssignmentIsUniform = true) - }) - } - - @Test - def testRoundRobinPartitionAssignorUnbalancedStaticSubscriptions() { - val assignor = new RoundRobinAssignor - val minConsumerCount = 5 - - /** test unbalanced static subscription scenarios */ - (1 to PartitionAssignorTest.TestCaseCount).foreach (testCase => { - val consumerCount = minConsumerCount.max(TestUtils.random.nextInt(PartitionAssignorTest.MaxConsumerCount + 1)) - val topicCount = 10 - - val topicPartitionCounts = Map((1 to topicCount).map(topic => { - ("topic-" + topic, 10) - }).toSeq:_*) - - val subscriptions = Map((1 to consumerCount).map(consumer => { - // Exclude some topics from some consumers - val topicRange = (1 to topicCount - consumer % minConsumerCount) - val streamCounts = Map(topicRange.map(topic => { - ("topic-" + topic, 3) - }).toSeq:_*) - ("g1c" + consumer, StaticSubscriptionInfo(streamCounts)) - }).toSeq:_*) - val scenario = Scenario("g1", topicPartitionCounts, subscriptions) - val zkUtils = PartitionAssignorTest.setupZkClientMock(scenario) - EasyMock.replay(zkUtils.zkClient) - PartitionAssignorTest.assignAndVerify(scenario, assignor, zkUtils) - }) - } - - @Test - def testRangePartitionAssignor() { - val assignor = new RangeAssignor - (1 to PartitionAssignorTest.TestCaseCount).foreach { _ => - val consumerCount = 1.max(TestUtils.random.nextInt(PartitionAssignorTest.MaxConsumerCount + 1)) - val topicCount = PartitionAssignorTest.MinTopicCount.max(TestUtils.random.nextInt(PartitionAssignorTest.MaxTopicCount + 1)) - - val topicPartitionCounts = Map((1 to topicCount).map(topic => { - ("topic-" + topic, PartitionAssignorTest.MinPartitionCount.max(TestUtils.random.nextInt(PartitionAssignorTest.MaxPartitionCount))) - }):_*) - - val subscriptions = Map((1 to consumerCount).map { consumer => - val streamCounts = Map((1 to topicCount).map(topic => { - val streamCount = 1.max(TestUtils.random.nextInt(PartitionAssignorTest.MaxStreamCount + 1)) - ("topic-" + topic, streamCount) - }):_*) - ("g1c" + consumer, StaticSubscriptionInfo(streamCounts)) - }:_*) - val scenario = Scenario("g1", topicPartitionCounts, subscriptions) - val zkUtils = PartitionAssignorTest.setupZkClientMock(scenario) - EasyMock.replay(zkUtils.zkClient) - - PartitionAssignorTest.assignAndVerify(scenario, assignor, zkUtils) - } - } -} - -private object PartitionAssignorTest extends Logging { - - private val TestCaseCount = 3 - private val MaxConsumerCount = 10 - private val MaxStreamCount = 8 - private val MaxTopicCount = 100 - private val MinTopicCount = 0 - private val MaxPartitionCount = 120 - private val MinPartitionCount = 8 - - private trait SubscriptionInfo { - def registrationString: String - } - - private case class StaticSubscriptionInfo(streamCounts: Map[String, Int]) extends SubscriptionInfo { - def registrationString = - Json.legacyEncodeAsString(Map("version" -> 1, - "subscription" -> streamCounts, - "pattern" -> "static", - "timestamp" -> 1234.toString)) - - override def toString = { - "Stream counts: " + streamCounts - } - } - - private case class WildcardSubscriptionInfo(streamCount: Int, regex: String, isWhitelist: Boolean) - extends SubscriptionInfo { - def registrationString = - Json.legacyEncodeAsString(Map("version" -> 1, - "subscription" -> Map(regex -> streamCount), - "pattern" -> (if (isWhitelist) "white_list" else "black_list"))) - - override def toString = { - "\"%s\":%d (%s)".format(regex, streamCount, if (isWhitelist) "whitelist" else "blacklist") - } - } - - private case class Scenario(group: String, - topicPartitionCounts: Map[String, Int], - /* consumerId -> SubscriptionInfo */ - subscriptions: Map[String, SubscriptionInfo]) { - override def toString = { - "\n" + - "Group : %s\n".format(group) + - "Topic partition counts : %s\n".format(topicPartitionCounts) + - "Consumer assignment : %s\n".format(subscriptions) - } - } - - private def setupZkClientMock(scenario: Scenario) = { - val consumers = java.util.Arrays.asList(scenario.subscriptions.keys.toSeq:_*) - - val zkClient = EasyMock.createStrictMock(classOf[ZkClient]) - val zkUtils = ZkUtils(zkClient, false) - EasyMock.checkOrder(zkClient, false) - - EasyMock.expect(zkClient.getChildren("/consumers/%s/ids".format(scenario.group))).andReturn(consumers) - EasyMock.expectLastCall().anyTimes() - - scenario.subscriptions.foreach { case(consumerId, subscriptionInfo) => - EasyMock.expect(zkClient.readData("/consumers/%s/ids/%s".format(scenario.group, consumerId), new Stat())) - .andReturn(subscriptionInfo.registrationString) - EasyMock.expectLastCall().anyTimes() - } - - scenario.topicPartitionCounts.foreach { case(topic, partitionCount) => - val replicaAssignment = Map((0 until partitionCount).map(partition => (partition.toString, Seq(0))):_*) - EasyMock.expect(zkClient.readData("/brokers/topics/%s".format(topic), new Stat())) - .andReturn(zkUtils.replicaAssignmentZkData(replicaAssignment)) - EasyMock.expectLastCall().anyTimes() - } - - EasyMock.expect(zkUtils.zkClient.getChildren("/brokers/topics")).andReturn( - java.util.Arrays.asList(scenario.topicPartitionCounts.keys.toSeq:_*)) - EasyMock.expectLastCall().anyTimes() - - zkUtils - } - - private def assignAndVerify(scenario: Scenario, assignor: PartitionAssignor, zkUtils: ZkUtils, - verifyAssignmentIsUniform: Boolean = false) { - val assignments = scenario.subscriptions.map { case (consumer, _) => - val ctx = new AssignmentContext("g1", consumer, excludeInternalTopics = true, zkUtils) - assignor.assign(ctx).get(consumer) - } - - // check for uniqueness (i.e., any partition should be assigned to exactly one consumer stream) - val globalAssignment = collection.mutable.Map[TopicAndPartition, ConsumerThreadId]() - assignments.foreach(assignment => { - assignment.foreach { case(topicPartition, owner) => - val previousOwnerOpt = globalAssignment.put(topicPartition, owner) - assertTrue("Scenario %s: %s is assigned to two owners.".format(scenario, topicPartition), previousOwnerOpt.isEmpty) - } - }) - - // check for coverage (i.e., all given partitions are owned) - val assignedPartitions = globalAssignment.keySet - val givenPartitions = scenario.topicPartitionCounts.flatMap{ case (topic, partitionCount) => - (0 until partitionCount).map(partition => TopicAndPartition(topic, partition)) - }.toSet - assertTrue("Scenario %s: the list of given partitions and assigned partitions are different.".format(scenario), - givenPartitions == assignedPartitions) - - // check for uniform assignment - if (verifyAssignmentIsUniform) { - val partitionCountForStream = partitionCountPerStream(globalAssignment) - if (partitionCountForStream.nonEmpty) { - val maxCount = partitionCountForStream.valuesIterator.max - val minCount = partitionCountForStream.valuesIterator.min - assertTrue("Scenario %s: assignment is not uniform (partition counts per stream are in the range [%d, %d])" - .format(scenario, minCount, maxCount), (maxCount - minCount) <= 1) - } - } - } - - /** For each consumer stream, count the number of partitions that it owns. */ - private def partitionCountPerStream(assignment: collection.Map[TopicAndPartition, ConsumerThreadId]) = { - val ownedCounts = collection.mutable.Map[ConsumerThreadId, Int]() - assignment.foreach { case (_, owner) => - val updatedCount = ownedCounts.getOrElse(owner, 0) + 1 - ownedCounts.put(owner, updatedCount) - } - ownedCounts - } -} - diff --git a/core/src/test/scala/unit/kafka/consumer/ZookeeperConsumerConnectorTest.scala b/core/src/test/scala/unit/kafka/consumer/ZookeeperConsumerConnectorTest.scala deleted file mode 100644 index 91d0af451ba05..0000000000000 --- a/core/src/test/scala/unit/kafka/consumer/ZookeeperConsumerConnectorTest.scala +++ /dev/null @@ -1,448 +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.consumer - -import java.util.Properties - -import org.junit.Assert._ -import kafka.common.MessageStreamsExistException -import kafka.integration.KafkaServerTestHarness -import kafka.javaapi.consumer.ConsumerRebalanceListener -import kafka.serializer._ -import kafka.server._ -import kafka.utils.TestUtils._ -import kafka.utils._ -import org.apache.kafka.common.record.CompressionType -import org.apache.kafka.common.security.JaasUtils -import org.apache.log4j.{Level, Logger} -import org.junit.{After, Before, Test} - -import scala.collection._ - -@deprecated("This test has been deprecated and it will be removed in a future release", "0.10.0.0") -class ZookeeperConsumerConnectorTest extends KafkaServerTestHarness with Logging { - - val RebalanceBackoffMs = 5000 - var dirs : ZKGroupTopicDirs = null - val numNodes = 2 - val numParts = 2 - val topic = "topic1" - val overridingProps = new Properties() - overridingProps.put(KafkaConfig.NumPartitionsProp, numParts.toString) - var zkUtils: ZkUtils = null - - override def generateConfigs = - TestUtils.createBrokerConfigs(numNodes, zkConnect).map(KafkaConfig.fromProps(_, overridingProps)) - - val group = "group1" - val consumer0 = "consumer0" - val consumer1 = "consumer1" - val consumer2 = "consumer2" - val consumer3 = "consumer3" - val nMessages = 2 - - @Before - override def setUp() { - super.setUp() - zkUtils = ZkUtils(zkConnect, zkSessionTimeout, zkConnectionTimeout, zkAclsEnabled.getOrElse(JaasUtils.isZkSecurityEnabled)) - dirs = new ZKGroupTopicDirs(group, topic) - } - - @After - override def tearDown() { - if (zkUtils != null) - CoreUtils.swallow(zkUtils.close(), this) - super.tearDown() - } - - @Test - def testBasic() { - val requestHandlerLogger = Logger.getLogger(classOf[KafkaRequestHandler]) - requestHandlerLogger.setLevel(Level.FATAL) - - // test consumer timeout logic - val consumerConfig0 = new ConsumerConfig( - TestUtils.createConsumerProperties(zkConnect, group, consumer0)) { - override val consumerTimeoutMs = 200 - } - val zkConsumerConnector0 = new ZookeeperConsumerConnector(consumerConfig0, true) - val topicMessageStreams0 = zkConsumerConnector0.createMessageStreams(Map(topic -> 1), new StringDecoder(), new StringDecoder()) - - // no messages to consume, we should hit timeout; - // also the iterator should support re-entrant, so loop it twice - for (_ <- 0 until 2) { - try { - getMessages(topicMessageStreams0, nMessages * 2) - fail("should get an exception") - } catch { - case _: ConsumerTimeoutException => // this is ok - } - } - - zkConsumerConnector0.shutdown - - // send some messages to each broker - val sentMessages1 = produceMessages(nMessages, acks = 0) ++ produceMessages(nMessages, acks = 1) - - // wait to make sure the topic and partition have a leader for the successful case - waitUntilLeaderIsElectedOrChanged(zkClient, topic, 0) - waitUntilLeaderIsElectedOrChanged(zkClient, topic, 1) - - TestUtils.waitUntilMetadataIsPropagated(servers, topic, 0) - TestUtils.waitUntilMetadataIsPropagated(servers, topic, 1) - - // create a consumer - val consumerConfig1 = new ConsumerConfig(TestUtils.createConsumerProperties(zkConnect, group, consumer1)) - val zkConsumerConnector1 = new ZookeeperConsumerConnector(consumerConfig1, true) - val topicMessageStreams1 = zkConsumerConnector1.createMessageStreams(Map(topic -> 1), new StringDecoder(), new StringDecoder()) - - val receivedMessages1 = getMessages(topicMessageStreams1, nMessages * 2) - assertEquals(sentMessages1.sorted, receivedMessages1.sorted) - - // also check partition ownership - val actual_1 = getZKChildrenValues(dirs.consumerOwnerDir) - val expected_1 = List( ("0", "group1_consumer1-0"), - ("1", "group1_consumer1-0")) - assertEquals(expected_1, actual_1) - - // commit consumed offsets - zkConsumerConnector1.commitOffsets(true) - - // create a consumer - val consumerConfig2 = new ConsumerConfig(TestUtils.createConsumerProperties(zkConnect, group, consumer2)) { - override val rebalanceBackoffMs = RebalanceBackoffMs - } - 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 = produceMessages(nMessages, acks = 0) ++ produceMessages(nMessages, acks = 1) - - waitUntilLeaderIsElectedOrChanged(zkClient, topic, 0) - waitUntilLeaderIsElectedOrChanged(zkClient, topic, 1) - - val receivedMessages2 = getMessages(topicMessageStreams1, nMessages) ++ getMessages(topicMessageStreams2, nMessages) - assertEquals(sentMessages2.sorted, receivedMessages2.sorted) - - // also check partition ownership - val actual_2 = getZKChildrenValues(dirs.consumerOwnerDir) - val expected_2 = List( ("0", "group1_consumer1-0"), - ("1", "group1_consumer2-0")) - assertEquals(expected_2, actual_2) - - // create a consumer with empty map - val consumerConfig3 = new ConsumerConfig( - TestUtils.createConsumerProperties(zkConnect, group, consumer3)) - val zkConsumerConnector3 = new ZookeeperConsumerConnector(consumerConfig3, true) - zkConsumerConnector3.createMessageStreams(new mutable.HashMap[String, Int]()) - // send some messages to each broker - val sentMessages3 = produceMessages(nMessages, acks = 0) ++ produceMessages(nMessages, acks = 1) - - waitUntilLeaderIsElectedOrChanged(zkClient, topic, 0) - waitUntilLeaderIsElectedOrChanged(zkClient, topic, 1) - - val receivedMessages3 = getMessages(topicMessageStreams1, nMessages) ++ getMessages(topicMessageStreams2, nMessages) - assertEquals(sentMessages3.sorted, receivedMessages3.sorted) - - // also check partition ownership - val actual_3 = getZKChildrenValues(dirs.consumerOwnerDir) - assertEquals(expected_2, actual_3) - - // call createMesssageStreams twice should throw MessageStreamsExistException - try { - zkConsumerConnector3.createMessageStreams(new mutable.HashMap[String, Int]()) - fail("Should fail with MessageStreamsExistException") - } catch { - case _: MessageStreamsExistException => // expected - } - - zkConsumerConnector1.shutdown - zkConsumerConnector2.shutdown - zkConsumerConnector3.shutdown - info("all consumer connectors stopped") - requestHandlerLogger.setLevel(Level.ERROR) - } - - private def produceMessages(numMessages: Int, acks: Int = -1, - compressionType: CompressionType = CompressionType.NONE): Seq[String] = { - TestUtils.generateAndProduceMessages(servers, topic, numMessages, acks, compressionType) - } - - @Test - def testCompression() { - val requestHandlerLogger = Logger.getLogger(classOf[kafka.server.KafkaRequestHandler]) - requestHandlerLogger.setLevel(Level.FATAL) - - // send some messages to each broker - val sentMessages1 = produceMessages(nMessages, acks = 0, CompressionType.GZIP) ++ - produceMessages(nMessages, acks = 1, CompressionType.GZIP) - - waitUntilLeaderIsElectedOrChanged(zkClient, topic, 0) - waitUntilLeaderIsElectedOrChanged(zkClient, topic, 1) - - TestUtils.waitUntilMetadataIsPropagated(servers, topic, 0) - TestUtils.waitUntilMetadataIsPropagated(servers, topic, 1) - - // create a consumer - val consumerConfig1 = new ConsumerConfig( - TestUtils.createConsumerProperties(zkConnect, group, consumer1)) - val zkConsumerConnector1 = new ZookeeperConsumerConnector(consumerConfig1, true) - val topicMessageStreams1 = zkConsumerConnector1.createMessageStreams(Map(topic -> 1), new StringDecoder(), new StringDecoder()) - val receivedMessages1 = getMessages(topicMessageStreams1, nMessages * 2) - assertEquals(sentMessages1.sorted, receivedMessages1.sorted) - - // also check partition ownership - val actual_1 = getZKChildrenValues(dirs.consumerOwnerDir) - val expected_1 = List( ("0", "group1_consumer1-0"), - ("1", "group1_consumer1-0")) - assertEquals(expected_1, actual_1) - - // commit consumed offsets - zkConsumerConnector1.commitOffsets(true) - - // create a consumer - val consumerConfig2 = new ConsumerConfig(TestUtils.createConsumerProperties(zkConnect, group, consumer2)) { - override val rebalanceBackoffMs = RebalanceBackoffMs - } - 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 = produceMessages(nMessages, acks = 0, CompressionType.GZIP) ++ - produceMessages(nMessages, acks = 1, CompressionType.GZIP) - - waitUntilLeaderIsElectedOrChanged(zkClient, topic, 0) - waitUntilLeaderIsElectedOrChanged(zkClient, topic, 1) - - val receivedMessages2 = getMessages(topicMessageStreams1, nMessages) ++ getMessages(topicMessageStreams2, nMessages) - assertEquals(sentMessages2.sorted, receivedMessages2.sorted) - - // also check partition ownership - val actual_2 = getZKChildrenValues(dirs.consumerOwnerDir) - val expected_2 = List( ("0", "group1_consumer1-0"), - ("1", "group1_consumer2-0")) - assertEquals(expected_2, actual_2) - - // create a consumer with empty map - val consumerConfig3 = new ConsumerConfig( - TestUtils.createConsumerProperties(zkConnect, group, consumer3)) - val zkConsumerConnector3 = new ZookeeperConsumerConnector(consumerConfig3, true) - zkConsumerConnector3.createMessageStreams(new mutable.HashMap[String, Int](), new StringDecoder(), new StringDecoder()) - // send some messages to each broker - val sentMessages3 = produceMessages(nMessages, acks = 0, CompressionType.GZIP) ++ - produceMessages(nMessages, acks = 1, CompressionType.GZIP) - - waitUntilLeaderIsElectedOrChanged(zkClient, topic, 0) - waitUntilLeaderIsElectedOrChanged(zkClient, topic, 1) - - val receivedMessages3 = getMessages(topicMessageStreams1, nMessages) ++ getMessages(topicMessageStreams2, nMessages) - assertEquals(sentMessages3.sorted, receivedMessages3.sorted) - - // also check partition ownership - val actual_3 = getZKChildrenValues(dirs.consumerOwnerDir) - assertEquals(expected_2, actual_3) - - zkConsumerConnector1.shutdown - zkConsumerConnector2.shutdown - zkConsumerConnector3.shutdown - info("all consumer connectors stopped") - requestHandlerLogger.setLevel(Level.ERROR) - } - - @Test - def testCompressionSetConsumption() { - // send some messages to each broker - val sentMessages = produceMessages(200, acks = 0, CompressionType.GZIP) ++ - produceMessages(200, acks = 1, CompressionType.GZIP) - - TestUtils.waitUntilMetadataIsPropagated(servers, topic, 0) - TestUtils.waitUntilMetadataIsPropagated(servers, topic, 1) - - val consumerConfig1 = new ConsumerConfig(TestUtils.createConsumerProperties(zkConnect, group, consumer0)) - val zkConsumerConnector1 = new ZookeeperConsumerConnector(consumerConfig1, true) - val topicMessageStreams1 = zkConsumerConnector1.createMessageStreams(Map(topic -> 1), new StringDecoder(), new StringDecoder()) - val receivedMessages = getMessages(topicMessageStreams1, 400) - assertEquals(sentMessages.sorted, receivedMessages.sorted) - - // also check partition ownership - val actual_2 = getZKChildrenValues(dirs.consumerOwnerDir) - val expected_2 = List( ("0", "group1_consumer0-0"), - ("1", "group1_consumer0-0")) - assertEquals(expected_2, actual_2) - - 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 = produceMessages(nMessages, acks = 0) ++ produceMessages(nMessages, acks = 1) - - TestUtils.waitUntilMetadataIsPropagated(servers, topic, 0) - TestUtils.waitUntilMetadataIsPropagated(servers, topic, 1) - - val consumerConfig = new ConsumerConfig(TestUtils.createConsumerProperties(zkConnect, group, consumer1)) - - waitUntilLeaderIsElectedOrChanged(zkClient, topic, 0) - waitUntilLeaderIsElectedOrChanged(zkClient, topic, 1) - - val zkConsumerConnector = - new ZookeeperConsumerConnector(consumerConfig, true) - val topicMessageStreams = - zkConsumerConnector.createMessageStreams(Map(topic -> 1), new StringDecoder(), new StringDecoder()) - - var receivedMessages: List[String] = Nil - for (messageStreams <- topicMessageStreams.values) { - for (messageStream <- messageStreams) { - val iterator = messageStream.iterator - for (_ <- 0 until nMessages * 2) { - assertTrue(iterator.hasNext()) - val message = iterator.next().message - receivedMessages ::= message - debug("received message: " + message) - } - } - } - assertEquals(sentMessages.sorted, receivedMessages.sorted) - - zkConsumerConnector.shutdown() - requestHandlerLogger.setLevel(Level.ERROR) - } - - @Test - def testLeaderSelectionForPartition() { - val zkUtils = ZkUtils(zkConnect, 6000, 30000, false) - - // create topic topic1 with 1 partition on broker 0 - createTopic(topic, numPartitions = 1, replicationFactor = 1) - - // send some messages to each broker - val sentMessages1 = produceMessages(nMessages) - - // create a consumer - val consumerConfig1 = new ConsumerConfig(TestUtils.createConsumerProperties(zkConnect, group, consumer1)) - val zkConsumerConnector1 = new ZookeeperConsumerConnector(consumerConfig1, true) - val topicMessageStreams1 = zkConsumerConnector1.createMessageStreams(Map(topic -> 1), new StringDecoder(), new StringDecoder()) - val topicRegistry = zkConsumerConnector1.getTopicRegistry - assertEquals(1, topicRegistry.map(r => r._1).size) - assertEquals(topic, topicRegistry.map(r => r._1).head) - val topicsAndPartitionsInRegistry = topicRegistry.map(r => (r._1, r._2.map(p => p._2))) - val brokerPartition = topicsAndPartitionsInRegistry.head._2.head - assertEquals(0, brokerPartition.partitionId) - - // also check partition ownership - val actual_1 = getZKChildrenValues(dirs.consumerOwnerDir) - val expected_1 = List( ("0", "group1_consumer1-0")) - assertEquals(expected_1, actual_1) - - val receivedMessages1 = getMessages(topicMessageStreams1, nMessages) - assertEquals(sentMessages1, receivedMessages1) - zkConsumerConnector1.shutdown() - zkUtils.close() - } - - @Test - def testConsumerRebalanceListener() { - // Send messages to create topic - produceMessages(nMessages, acks = 0) - produceMessages(nMessages, acks = 1) - - 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()) - - // 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) - 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 = zkUtils.getChildren(path).sorted - children.map(partition => - (partition, zkUtils.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/ControllerIntegrationTest.scala b/core/src/test/scala/unit/kafka/controller/ControllerIntegrationTest.scala index 88fe82bab0ef9..f4f40395322bb 100644 --- a/core/src/test/scala/unit/kafka/controller/ControllerIntegrationTest.scala +++ b/core/src/test/scala/unit/kafka/controller/ControllerIntegrationTest.scala @@ -309,7 +309,7 @@ class ControllerIntegrationTest extends ZooKeeperTestHarness { } private def waitUntilControllerEpoch(epoch: Int, message: String): Unit = { - TestUtils.waitUntilTrue(() => zkClient.getControllerEpoch.get._1 == epoch, message) + TestUtils.waitUntilTrue(() => zkClient.getControllerEpoch.map(_._1).contains(epoch) , message) } private def waitForPartitionState(tp: TopicPartition, diff --git a/core/src/test/scala/unit/kafka/coordinator/group/GroupCoordinatorTest.scala b/core/src/test/scala/unit/kafka/coordinator/group/GroupCoordinatorTest.scala index 608d7cc997f15..efa44fa4ae87b 100644 --- a/core/src/test/scala/unit/kafka/coordinator/group/GroupCoordinatorTest.scala +++ b/core/src/test/scala/unit/kafka/coordinator/group/GroupCoordinatorTest.scala @@ -527,11 +527,29 @@ class GroupCoordinatorTest extends JUnitSuite { heartbeatResult = heartbeat(groupId, firstMemberId, firstGenerationId) assertEquals(Errors.REBALANCE_IN_PROGRESS, heartbeatResult) - // now timeout the rebalance, which should kick the unjoined member out of the group - // and let the rebalance finish with only the new member + // now timeout the rebalance timer.advanceClock(500) val otherJoinResult = await(otherJoinFuture, DefaultSessionTimeout+100) + val otherMemberId = otherJoinResult.memberId + val otherGenerationId = otherJoinResult.generationId + EasyMock.reset(replicaManager) + val syncResult = syncGroupLeader(groupId, otherGenerationId, otherMemberId, Map(otherMemberId -> Array[Byte]())) + assertEquals(Errors.NONE, syncResult._2) + + // the unjoined member should be kicked out from the group assertEquals(Errors.NONE, otherJoinResult.error) + EasyMock.reset(replicaManager) + heartbeatResult = heartbeat(groupId, firstMemberId, firstGenerationId) + assertEquals(Errors.UNKNOWN_MEMBER_ID, heartbeatResult) + + // the joined member should get heart beat response with no error. Let the new member keep heartbeating for a while + // to verify that no new rebalance is triggered unexpectedly + for ( _ <- 1 to 20) { + timer.advanceClock(500) + EasyMock.reset(replicaManager) + heartbeatResult = heartbeat(groupId, otherMemberId, otherGenerationId) + assertEquals(Errors.NONE, heartbeatResult) + } } @Test diff --git a/core/src/test/scala/unit/kafka/coordinator/group/GroupMetadataManagerTest.scala b/core/src/test/scala/unit/kafka/coordinator/group/GroupMetadataManagerTest.scala index b358c4e8279bb..99aa666624b25 100644 --- a/core/src/test/scala/unit/kafka/coordinator/group/GroupMetadataManagerTest.scala +++ b/core/src/test/scala/unit/kafka/coordinator/group/GroupMetadataManagerTest.scala @@ -33,6 +33,7 @@ import org.easymock.{Capture, EasyMock, IAnswer} import org.junit.Assert.{assertEquals, assertFalse, assertTrue, assertNull} import org.junit.{Before, Test} import java.nio.ByteBuffer +import java.util.Collections import com.yammer.metrics.Metrics import com.yammer.metrics.core.Gauge @@ -639,6 +640,38 @@ class GroupMetadataManagerTest { assertEquals(None, groupMetadataManager.getGroup(groupId)) } + @Test + def testLoadGroupWithLargeGroupMetadataRecord() { + val groupMetadataTopicPartition = groupTopicPartition + val startOffset = 15L + val committedOffsets = Map( + new TopicPartition("foo", 0) -> 23L, + new TopicPartition("foo", 1) -> 455L, + new TopicPartition("bar", 0) -> 8992L + ) + + // create a GroupMetadata record larger then offsets.load.buffer.size (here at least 16 bytes larger) + val assignmentSize = OffsetConfig.DefaultLoadBufferSize + 16 + val memberId = "98098230493" + + val offsetCommitRecords = createCommittedOffsetRecords(committedOffsets) + val groupMetadataRecord = buildStableGroupRecordWithMember(generation = 15, + protocolType = "consumer", protocol = "range", memberId, assignmentSize) + val records = MemoryRecords.withRecords(startOffset, CompressionType.NONE, + offsetCommitRecords ++ Seq(groupMetadataRecord): _*) + + expectGroupMetadataLoad(groupMetadataTopicPartition, startOffset, records) + + EasyMock.replay(replicaManager) + + groupMetadataManager.loadGroupsAndOffsets(groupMetadataTopicPartition, _ => ()) + + val group = groupMetadataManager.getGroup(groupId).getOrElse(fail("Group was not loaded into the cache")) + committedOffsets.foreach { case (topicPartition, offset) => + assertEquals(Some(offset), group.offset(topicPartition).map(_.offset)) + } + } + @Test def testOffsetWriteAfterGroupRemoved(): Unit = { // this test case checks the following scenario: @@ -1413,6 +1446,61 @@ class GroupMetadataManagerTest { EasyMock.verify(replicaManager) } + @Test + def testLoadOffsetsWithEmptyControlBatch() { + val groupMetadataTopicPartition = groupTopicPartition + val startOffset = 15L + val generation = 15 + + val committedOffsets = Map( + new TopicPartition("foo", 0) -> 23L, + new TopicPartition("foo", 1) -> 455L, + new TopicPartition("bar", 0) -> 8992L + ) + + val offsetCommitRecords = createCommittedOffsetRecords(committedOffsets) + val groupMetadataRecord = buildEmptyGroupRecord(generation, protocolType) + val records = MemoryRecords.withRecords(startOffset, CompressionType.NONE, + offsetCommitRecords ++ Seq(groupMetadataRecord): _*) + + // Prepend empty control batch to valid records + val mockBatch = EasyMock.createMock(classOf[MutableRecordBatch]) + EasyMock.expect(mockBatch.iterator).andReturn(Collections.emptyIterator[Record]) + EasyMock.expect(mockBatch.isControlBatch).andReturn(true) + EasyMock.expect(mockBatch.isTransactional).andReturn(true) + EasyMock.expect(mockBatch.nextOffset).andReturn(16L) + EasyMock.replay(mockBatch) + val mockRecords = EasyMock.createMock(classOf[MemoryRecords]) + EasyMock.expect(mockRecords.batches).andReturn((Iterable[MutableRecordBatch](mockBatch) ++ records.batches.asScala).asJava).anyTimes() + EasyMock.expect(mockRecords.records).andReturn(records.records()).anyTimes() + EasyMock.expect(mockRecords.sizeInBytes()).andReturn(DefaultRecordBatch.RECORD_BATCH_OVERHEAD + records.sizeInBytes()).anyTimes() + EasyMock.replay(mockRecords) + + val logMock = EasyMock.mock(classOf[Log]) + EasyMock.expect(logMock.logStartOffset).andReturn(startOffset).anyTimes() + EasyMock.expect(logMock.read(EasyMock.eq(startOffset), EasyMock.anyInt(), EasyMock.eq(None), + EasyMock.eq(true), EasyMock.eq(IsolationLevel.READ_UNCOMMITTED))) + .andReturn(FetchDataInfo(LogOffsetMetadata(startOffset), mockRecords)) + EasyMock.expect(replicaManager.getLog(groupMetadataTopicPartition)).andStubReturn(Some(logMock)) + EasyMock.expect(replicaManager.getLogEndOffset(groupMetadataTopicPartition)).andStubReturn(Some(18)) + EasyMock.replay(logMock) + EasyMock.replay(replicaManager) + + groupMetadataManager.loadGroupsAndOffsets(groupMetadataTopicPartition, _ => ()) + + // Empty control batch should not have caused the load to fail + val group = groupMetadataManager.getGroup(groupId).getOrElse(fail("Group was not loaded into the cache")) + assertEquals(groupId, group.groupId) + assertEquals(Empty, group.currentState) + assertEquals(generation, group.generationId) + assertEquals(Some(protocolType), group.protocolType) + assertNull(group.leaderOrNull) + assertNull(group.protocolOrNull) + committedOffsets.foreach { case (topicPartition, offset) => + assertEquals(Some(offset), group.offset(topicPartition).map(_.offset)) + } + } + private def appendAndCaptureCallback(): Capture[Map[TopicPartition, PartitionResponse] => Unit] = { val capturedArgument: Capture[Map[TopicPartition, PartitionResponse] => Unit] = EasyMock.newCapture() EasyMock.expect(replicaManager.appendRecords(EasyMock.anyLong(), @@ -1452,7 +1540,8 @@ class GroupMetadataManagerTest { private def buildStableGroupRecordWithMember(generation: Int, protocolType: String, protocol: String, - memberId: String): SimpleRecord = { + memberId: String, + assignmentSize: Int = 0): SimpleRecord = { val memberProtocols = List((protocol, Array.emptyByteArray)) val member = new MemberMetadata(memberId, groupId, "clientId", "clientHost", 30000, 10000, protocolType, memberProtocols) val group = GroupMetadata.loadGroup(groupId, Stable, generation, protocolType, protocol, @@ -1494,8 +1583,19 @@ class GroupMetadataManagerTest { EasyMock.expect(logMock.read(EasyMock.eq(startOffset), EasyMock.anyInt(), EasyMock.eq(None), EasyMock.eq(true), EasyMock.eq(IsolationLevel.READ_UNCOMMITTED))) .andReturn(FetchDataInfo(LogOffsetMetadata(startOffset), fileRecordsMock)) - EasyMock.expect(fileRecordsMock.readInto(EasyMock.anyObject(classOf[ByteBuffer]), EasyMock.anyInt())) - .andReturn(records.buffer) + + EasyMock.expect(fileRecordsMock.sizeInBytes()).andStubReturn(records.sizeInBytes) + + val bufferCapture = EasyMock.newCapture[ByteBuffer] + fileRecordsMock.readInto(EasyMock.capture(bufferCapture), EasyMock.anyInt()) + EasyMock.expectLastCall().andAnswer(new IAnswer[Unit] { + override def answer: Unit = { + val buffer = bufferCapture.getValue + buffer.put(records.buffer.duplicate) + buffer.flip() + } + }) + EasyMock.replay(fileRecordsMock) endOffset diff --git a/core/src/test/scala/unit/kafka/coordinator/transaction/ProducerIdManagerTest.scala b/core/src/test/scala/unit/kafka/coordinator/transaction/ProducerIdManagerTest.scala index 88aebd3ce498d..660e62309e7d6 100644 --- a/core/src/test/scala/unit/kafka/coordinator/transaction/ProducerIdManagerTest.scala +++ b/core/src/test/scala/unit/kafka/coordinator/transaction/ProducerIdManagerTest.scala @@ -16,8 +16,8 @@ */ package kafka.coordinator.transaction -import kafka.common.KafkaException import kafka.zk.KafkaZkClient +import org.apache.kafka.common.KafkaException import org.easymock.{Capture, EasyMock, IAnswer} import org.junit.{After, Test} import org.junit.Assert._ diff --git a/core/src/test/scala/unit/kafka/coordinator/transaction/TransactionCoordinatorConcurrencyTest.scala b/core/src/test/scala/unit/kafka/coordinator/transaction/TransactionCoordinatorConcurrencyTest.scala index 6168077248ba4..060e07e732788 100644 --- a/core/src/test/scala/unit/kafka/coordinator/transaction/TransactionCoordinatorConcurrencyTest.scala +++ b/core/src/test/scala/unit/kafka/coordinator/transaction/TransactionCoordinatorConcurrencyTest.scala @@ -22,21 +22,19 @@ import kafka.coordinator.AbstractCoordinatorConcurrencyTest import kafka.coordinator.AbstractCoordinatorConcurrencyTest._ import kafka.coordinator.transaction.TransactionCoordinatorConcurrencyTest._ import kafka.log.Log -import kafka.server.{ DelayedOperationPurgatory, FetchDataInfo, KafkaConfig, LogOffsetMetadata, MetadataCache } +import kafka.server.{DelayedOperationPurgatory, FetchDataInfo, KafkaConfig, LogOffsetMetadata, MetadataCache} import kafka.utils.timer.MockTimer -import kafka.utils.{ Pool, TestUtils} - -import org.apache.kafka.clients.{ ClientResponse, NetworkClient } -import org.apache.kafka.common.{ Node, TopicPartition } +import kafka.utils.{Pool, TestUtils} +import org.apache.kafka.clients.{ClientResponse, NetworkClient} +import org.apache.kafka.common.{Node, TopicPartition} import org.apache.kafka.common.internals.Topic.TRANSACTION_STATE_TOPIC_NAME -import org.apache.kafka.common.protocol.{ ApiKeys, Errors } -import org.apache.kafka.common.record.{ CompressionType, FileRecords, MemoryRecords, SimpleRecord } +import org.apache.kafka.common.protocol.{ApiKeys, Errors} +import org.apache.kafka.common.record.{CompressionType, FileRecords, MemoryRecords, SimpleRecord} import org.apache.kafka.common.requests._ -import org.apache.kafka.common.utils.{ LogContext, MockTime } - -import org.easymock.EasyMock +import org.apache.kafka.common.utils.{LogContext, MockTime} +import org.easymock.{EasyMock, IAnswer} import org.junit.Assert._ -import org.junit.{ After, Before, Test } +import org.junit.{After, Before, Test} import scala.collection.Map import scala.collection.mutable @@ -260,8 +258,18 @@ class TransactionCoordinatorConcurrencyTest extends AbstractCoordinatorConcurren EasyMock.expect(logMock.read(EasyMock.eq(startOffset), EasyMock.anyInt(), EasyMock.eq(None), EasyMock.eq(true), EasyMock.eq(IsolationLevel.READ_UNCOMMITTED))) .andReturn(FetchDataInfo(LogOffsetMetadata(startOffset), fileRecordsMock)) - EasyMock.expect(fileRecordsMock.readInto(EasyMock.anyObject(classOf[ByteBuffer]), EasyMock.anyInt())) - .andReturn(records.buffer) + + EasyMock.expect(fileRecordsMock.sizeInBytes()).andStubReturn(records.sizeInBytes) + + val bufferCapture = EasyMock.newCapture[ByteBuffer] + fileRecordsMock.readInto(EasyMock.capture(bufferCapture), EasyMock.anyInt()) + EasyMock.expectLastCall().andAnswer(new IAnswer[Unit] { + override def answer: Unit = { + val buffer = bufferCapture.getValue + buffer.put(records.buffer.duplicate) + buffer.flip() + } + }) EasyMock.replay(logMock, fileRecordsMock) synchronized { diff --git a/core/src/test/scala/unit/kafka/coordinator/transaction/TransactionStateManagerTest.scala b/core/src/test/scala/unit/kafka/coordinator/transaction/TransactionStateManagerTest.scala index 20dfaa6ec1e4a..ff2cbcfe8880a 100644 --- a/core/src/test/scala/unit/kafka/coordinator/transaction/TransactionStateManagerTest.scala +++ b/core/src/test/scala/unit/kafka/coordinator/transaction/TransactionStateManagerTest.scala @@ -419,56 +419,57 @@ class TransactionStateManagerTest { def shouldRemoveCompleteCommmitExpiredTransactionalIds(): Unit = { setupAndRunTransactionalIdExpiration(Errors.NONE, CompleteCommit) verifyMetadataDoesntExist(transactionalId1) - verifyMetadataDoesExist(transactionalId2) + verifyMetadataDoesExistAndIsUsable(transactionalId2) } @Test def shouldRemoveCompleteAbortExpiredTransactionalIds(): Unit = { setupAndRunTransactionalIdExpiration(Errors.NONE, CompleteAbort) verifyMetadataDoesntExist(transactionalId1) - verifyMetadataDoesExist(transactionalId2) + verifyMetadataDoesExistAndIsUsable(transactionalId2) } @Test def shouldRemoveEmptyExpiredTransactionalIds(): Unit = { setupAndRunTransactionalIdExpiration(Errors.NONE, Empty) verifyMetadataDoesntExist(transactionalId1) - verifyMetadataDoesExist(transactionalId2) + verifyMetadataDoesExistAndIsUsable(transactionalId2) } @Test def shouldNotRemoveExpiredTransactionalIdsIfLogAppendFails(): Unit = { setupAndRunTransactionalIdExpiration(Errors.NOT_ENOUGH_REPLICAS, CompleteAbort) - verifyMetadataDoesExist(transactionalId1) - verifyMetadataDoesExist(transactionalId2) + verifyMetadataDoesExistAndIsUsable(transactionalId1) + verifyMetadataDoesExistAndIsUsable(transactionalId2) } @Test def shouldNotRemoveOngoingTransactionalIds(): Unit = { setupAndRunTransactionalIdExpiration(Errors.NONE, Ongoing) - verifyMetadataDoesExist(transactionalId1) - verifyMetadataDoesExist(transactionalId2) + verifyMetadataDoesExistAndIsUsable(transactionalId1) + verifyMetadataDoesExistAndIsUsable(transactionalId2) } @Test def shouldNotRemovePrepareAbortTransactionalIds(): Unit = { setupAndRunTransactionalIdExpiration(Errors.NONE, PrepareAbort) - verifyMetadataDoesExist(transactionalId1) - verifyMetadataDoesExist(transactionalId2) + verifyMetadataDoesExistAndIsUsable(transactionalId1) + verifyMetadataDoesExistAndIsUsable(transactionalId2) } @Test def shouldNotRemovePrepareCommitTransactionalIds(): Unit = { setupAndRunTransactionalIdExpiration(Errors.NONE, PrepareCommit) - verifyMetadataDoesExist(transactionalId1) - verifyMetadataDoesExist(transactionalId2) + verifyMetadataDoesExistAndIsUsable(transactionalId1) + verifyMetadataDoesExistAndIsUsable(transactionalId2) } - private def verifyMetadataDoesExist(transactionalId: String) = { + private def verifyMetadataDoesExistAndIsUsable(transactionalId: String) = { transactionManager.getTransactionState(transactionalId) match { case Left(errors) => fail("shouldn't have been any errors") case Right(None) => fail("metadata should have been removed") - case Right(Some(metadata)) => // ok + case Right(Some(metadata)) => + assertTrue("metadata shouldn't be in a pending state", metadata.transactionMetadata.pendingState.isEmpty) } } @@ -585,9 +586,18 @@ class TransactionStateManagerTest { EasyMock.expect(logMock.read(EasyMock.eq(startOffset), EasyMock.anyInt(), EasyMock.eq(None), EasyMock.eq(true), EasyMock.eq(IsolationLevel.READ_UNCOMMITTED))) .andReturn(FetchDataInfo(LogOffsetMetadata(startOffset), fileRecordsMock)) - EasyMock.expect(fileRecordsMock.readInto(EasyMock.anyObject(classOf[ByteBuffer]), EasyMock.anyInt())) - .andReturn(records.buffer) + EasyMock.expect(fileRecordsMock.sizeInBytes()).andStubReturn(records.sizeInBytes) + + val bufferCapture = EasyMock.newCapture[ByteBuffer] + fileRecordsMock.readInto(EasyMock.capture(bufferCapture), EasyMock.anyInt()) + EasyMock.expectLastCall().andAnswer(new IAnswer[Unit] { + override def answer: Unit = { + val buffer = bufferCapture.getValue + buffer.put(records.buffer.duplicate) + buffer.flip() + } + }) EasyMock.replay(logMock, fileRecordsMock, replicaManager) } diff --git a/core/src/test/scala/unit/kafka/integration/AutoOffsetResetTest.scala b/core/src/test/scala/unit/kafka/integration/AutoOffsetResetTest.scala deleted file mode 100644 index 5abc3525091f2..0000000000000 --- a/core/src/test/scala/unit/kafka/integration/AutoOffsetResetTest.scala +++ /dev/null @@ -1,117 +0,0 @@ -/** - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package kafka.integration - -import kafka.utils.{Logging, ZKGroupTopicDirs} -import kafka.consumer.{Consumer, ConsumerConfig, ConsumerConnector, ConsumerTimeoutException} -import kafka.server._ -import kafka.utils.TestUtils -import kafka.utils.TestUtils.createNewProducer -import org.apache.kafka.clients.producer.ProducerRecord -import org.junit.{After, Before, Test} -import org.apache.log4j.{Level, Logger} -import org.junit.Assert._ - -@deprecated("This test has been deprecated and it will be removed in a future release", "0.10.0.0") -class AutoOffsetResetTest extends KafkaServerTestHarness with Logging { - - def generateConfigs = List(KafkaConfig.fromProps(TestUtils.createBrokerConfig(0, zkConnect))) - - val topic = "test_topic" - val group = "default_group" - val testConsumer = "consumer" - val NumMessages = 10 - val LargeOffset = 10000 - val SmallOffset = -1 - - 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) - super.tearDown - } - - @Test - def testResetToEarliestWhenOffsetTooHigh() = - assertEquals(NumMessages, resetAndConsume(NumMessages, "smallest", LargeOffset)) - - @Test - def testResetToEarliestWhenOffsetTooLow() = - assertEquals(NumMessages, resetAndConsume(NumMessages, "smallest", SmallOffset)) - - @Test - def testResetToLatestWhenOffsetTooHigh() = - assertEquals(0, resetAndConsume(NumMessages, "largest", LargeOffset)) - - @Test - def testResetToLatestWhenOffsetTooLow() = - assertEquals(0, resetAndConsume(NumMessages, "largest", SmallOffset)) - - /* Produce the given number of messages, create a consumer with the given offset policy, - * then reset the offset to the given value and consume until we get no new messages. - * Returns the count of messages received. - */ - def resetAndConsume(numMessages: Int, resetTo: String, offset: Long): Int = { - createTopic(topic, 1, 1) - - val producer = createNewProducer(TestUtils.getBrokerListStrFromServers(servers), retries = 5) - - val futures = (0 until numMessages).map(_ => producer.send(new ProducerRecord(topic, topic.getBytes, "test".getBytes))) - futures.foreach(_.get) - - // update offset in ZooKeeper for consumer to jump "forward" in time - val dirs = new ZKGroupTopicDirs(group, topic) - val consumerProps = TestUtils.createConsumerProperties(zkConnect, group, testConsumer) - consumerProps.put("auto.offset.reset", resetTo) - consumerProps.put("consumer.timeout.ms", "2000") - consumerProps.put("fetch.wait.max.ms", "0") - val consumerConfig = new ConsumerConfig(consumerProps) - - TestUtils.updateConsumerOffset(consumerConfig, dirs.consumerOffsetDir + "/" + "0", offset) - info("Updated consumer offset to " + offset) - - val consumerConnector: ConsumerConnector = Consumer.create(consumerConfig) - val messageStream = consumerConnector.createMessageStreams(Map(topic -> 1))(topic).head - - var received = 0 - val iter = messageStream.iterator - try { - for (_ <- 0 until numMessages) { - iter.next // will throw a timeout exception if the message isn't there - received += 1 - } - } catch { - case _: ConsumerTimeoutException => - info("consumer timed out after receiving " + received + " messages.") - } finally { - producer.close() - consumerConnector.shutdown - } - received - } - -} diff --git a/core/src/test/scala/unit/kafka/integration/FetcherTest.scala b/core/src/test/scala/unit/kafka/integration/FetcherTest.scala deleted file mode 100644 index 18373f2ffba78..0000000000000 --- a/core/src/test/scala/unit/kafka/integration/FetcherTest.scala +++ /dev/null @@ -1,99 +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 java.util.concurrent._ -import java.util.concurrent.atomic._ - -import org.junit.{After, Before, Test} - -import scala.collection._ -import org.junit.Assert._ -import kafka.cluster._ -import kafka.server._ -import kafka.consumer._ -import kafka.utils.{CoreUtils, TestUtils, ZkUtils} -import org.apache.kafka.common.security.JaasUtils - -@deprecated("This test has been deprecated and will be removed in a future release.", "0.11.0.0") -class FetcherTest extends KafkaServerTestHarness { - val numNodes = 1 - def generateConfigs = TestUtils.createBrokerConfigs(numNodes, zkConnect).map(KafkaConfig.fromProps) - - val messages = new mutable.HashMap[Int, Seq[Array[Byte]]] - val topic = "topic" - val queue = new LinkedBlockingQueue[FetchedDataChunk] - - var fetcher: ConsumerFetcherManager = null - var zkUtils: ZkUtils = null - - @Before - override def setUp() { - super.setUp - zkUtils = ZkUtils(zkConnect, zkSessionTimeout, zkConnectionTimeout, zkAclsEnabled.getOrElse(JaasUtils.isZkSecurityEnabled)) - - createTopic(topic, partitionReplicaAssignment = Map(0 -> Seq(configs.head.brokerId))) - - val cluster = new Cluster(servers.map { s => - new Broker(s.config.brokerId, "localhost", boundPort(s), listenerName, securityProtocol) - }) - - fetcher = new ConsumerFetcherManager("consumer1", new ConsumerConfig(TestUtils.createConsumerProperties("", "", "")), zkUtils) - fetcher.stopConnections() - val topicInfos = configs.map(_ => - new PartitionTopicInfo(topic, - 0, - queue, - new AtomicLong(0), - new AtomicLong(0), - new AtomicInteger(0), - "")) - fetcher.startConnections(topicInfos, cluster) - } - - @After - override def tearDown() { - fetcher.stopConnections() - if (zkUtils != null) - CoreUtils.swallow(zkUtils.close(), this) - super.tearDown - } - - @Test - def testFetcher() { - val perNode = 2 - var count = TestUtils.generateAndProduceMessages(servers, topic, perNode).size - - fetch(count) - assertQueueEmpty() - count = TestUtils.generateAndProduceMessages(servers, topic, perNode).size - fetch(count) - assertQueueEmpty() - } - - def assertQueueEmpty(): Unit = assertEquals(0, queue.size) - - def fetch(expected: Int) { - var count = 0 - while (count < expected) { - val chunk = queue.poll(2L, TimeUnit.SECONDS) - assertNotNull("Timed out waiting for data chunk " + (count + 1), chunk) - count += chunk.messages.size - } - } -} diff --git a/core/src/test/scala/unit/kafka/integration/KafkaServerTestHarness.scala b/core/src/test/scala/unit/kafka/integration/KafkaServerTestHarness.scala index 662d6d20c2dd1..0c97357ef6b12 100755 --- a/core/src/test/scala/unit/kafka/integration/KafkaServerTestHarness.scala +++ b/core/src/test/scala/unit/kafka/integration/KafkaServerTestHarness.scala @@ -20,7 +20,6 @@ package kafka.integration import java.io.File import java.util.Arrays -import kafka.common.KafkaException import kafka.server._ import kafka.utils.TestUtils import kafka.zk.ZooKeeperTestHarness @@ -30,7 +29,9 @@ import org.junit.{After, Before} import scala.collection.mutable.{ArrayBuffer, Buffer} import java.util.Properties +import org.apache.kafka.common.KafkaException import org.apache.kafka.common.network.ListenerName +import org.apache.kafka.common.utils.Time /** * A test harness that brings up some number of broker nodes @@ -82,6 +83,7 @@ abstract class KafkaServerTestHarness extends ZooKeeperTestHarness { protected def trustStoreFile: Option[File] = None protected def serverSaslProperties: Option[Properties] = None protected def clientSaslProperties: Option[Properties] = None + protected def brokerTime(brokerId: Int): Time = Time.SYSTEM @Before override def setUp() { @@ -96,7 +98,7 @@ abstract class KafkaServerTestHarness extends ZooKeeperTestHarness { // Add each broker to `servers` buffer as soon as it is created to ensure that brokers // are shutdown cleanly in tearDown even if a subsequent broker fails to start for (config <- configs) - servers += TestUtils.createServer(config) + servers += TestUtils.createServer(config, time = brokerTime(config.brokerId)) brokerList = TestUtils.bootstrapServers(servers, listenerName) alive = new Array[Boolean](servers.length) Arrays.fill(alive, true) 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 87ffdf1031a68..0000000000000 --- a/core/src/test/scala/unit/kafka/integration/TopicMetadataTest.scala +++ /dev/null @@ -1,289 +0,0 @@ -/** - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. -*/ - -package kafka.integration - -import kafka.api.TopicMetadataResponse -import kafka.client.ClientUtils -import kafka.cluster.BrokerEndPoint -import kafka.server.{KafkaConfig, KafkaServer, NotRunning} -import kafka.utils.TestUtils -import kafka.utils.TestUtils._ -import kafka.zk.ZooKeeperTestHarness -import org.apache.kafka.common.protocol.Errors -import org.junit.Assert._ -import org.junit.{Test, After, Before} - -class TopicMetadataTest extends ZooKeeperTestHarness { - private var server1: KafkaServer = null - private var adHocServers: Seq[KafkaServer] = Seq() - var brokerEndPoints: Seq[BrokerEndPoint] = null - var adHocConfigs: Seq[KafkaConfig] = null - val numConfigs: Int = 4 - - @Before - override def setUp() { - super.setUp() - val props = createBrokerConfigs(numConfigs, zkConnect) - val configs: Seq[KafkaConfig] = props.map(KafkaConfig.fromProps) - adHocConfigs = configs.takeRight(configs.size - 1) // Started and stopped by individual test cases - server1 = TestUtils.createServer(configs.head) - brokerEndPoints = Seq( - // We are using the Scala clients and they don't support SSL. Once we move to the Java ones, we should use - // `securityProtocol` instead of PLAINTEXT below - new BrokerEndPoint(server1.config.brokerId, server1.config.hostName, TestUtils.boundPort(server1)) - ) - } - - @After - override def tearDown() { - TestUtils.shutdownServers(adHocServers :+ server1) - super.tearDown() - } - - @Test - def testBasicTopicMetadata(): Unit = { - // 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(Errors.NONE, topicsMetadata.head.error) - assertEquals(Errors.NONE, topicsMetadata.head.partitionsMetadata.head.error) - 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(): Unit = { - // 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(Errors.NONE, topicsMetadata.head.error) - assertEquals(2, topicsMetadata.size) - assertEquals(Errors.NONE, topicsMetadata.head.partitionsMetadata.head.error) - assertEquals(Errors.NONE, topicsMetadata.last.partitionsMetadata.head.error) - 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(): Unit = { - // auto create topic - val topic = "testAutoCreateTopic" - var topicsMetadata = ClientUtils.fetchTopicMetadata(Set(topic), brokerEndPoints, "TopicMetadataTest-testAutoCreateTopic", - 2000,0).topicsMetadata - assertEquals(Errors.LEADER_NOT_AVAILABLE, topicsMetadata.head.error) - 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(Errors.NONE, topicsMetadata.head.error) - assertEquals(Errors.NONE, topicsMetadata.head.partitionsMetadata.head.error) - 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 testAutoCreateTopicWithInvalidReplication(): Unit = { - val adHocProps = createBrokerConfig(2, zkConnect) - // Set default replication higher than the number of live brokers - adHocProps.setProperty(KafkaConfig.DefaultReplicationFactorProp, "3") - // start adHoc brokers with replication factor too high - val adHocServer = createServer(new KafkaConfig(adHocProps)) - adHocServers = Seq(adHocServer) - // We are using the Scala clients and they don't support SSL. Once we move to the Java ones, we should use - // `securityProtocol` instead of PLAINTEXT below - val adHocEndpoint = new BrokerEndPoint(adHocServer.config.brokerId, adHocServer.config.hostName, - TestUtils.boundPort(adHocServer)) - - // auto create topic on "bad" endpoint - val topic = "testAutoCreateTopic" - val topicsMetadata = ClientUtils.fetchTopicMetadata(Set(topic), Seq(adHocEndpoint), "TopicMetadataTest-testAutoCreateTopic", - 2000, 0).topicsMetadata - assertEquals(Errors.INVALID_REPLICATION_FACTOR, topicsMetadata.head.error) - 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) - } - - @Test - def testAutoCreateTopicWithCollision(): Unit = { - // 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(Errors.LEADER_NOT_AVAILABLE, topicsMetadata.head.error) - assertEquals("Expecting metadata for topic2", topic2, topicsMetadata(1).topic) - assertEquals("Expecting InvalidTopicCode for topic2 metadata", Errors.INVALID_TOPIC_EXCEPTION, topicsMetadata(1).error) - - // 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(Errors.NONE, topicsMetadata.head.error) - assertEquals(Errors.NONE, topicsMetadata.head.partitionsMetadata.head.error) - 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) - } - - 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", - TestUtils.boundPort(x)) - } - - // 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", - TestUtils.boundPort(x))), - "TopicMetadataTest-testBasicTopicMetadata", 2000, 0) - metadata.topicsMetadata.nonEmpty && - metadata.topicsMetadata.head.partitionsMetadata.nonEmpty && - expectedIsr.sortBy(_.id) == metadata.topicsMetadata.head.partitionsMetadata.head.isr.sortBy(_.id) - }, - "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 - ""), 8000L) - }) - } - - @Test - def testIsrAfterBrokerShutDownAndJoinsBack(): Unit = { - val numBrokers = 2 //just 2 brokers are enough for the test - - // start adHoc brokers - adHocServers = adHocConfigs.take(numBrokers - 1).map(p => createServer(p)) - val allServers: Seq[KafkaServer] = Seq(server1) ++ adHocServers - - // create topic - val topic: String = "test" - adminZkClient.createTopic(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) - } - - 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(() => { - val foundMetadata = ClientUtils.fetchTopicMetadata( - Set.empty, - Seq(new BrokerEndPoint(x.config.brokerId, x.config.hostName, TestUtils.boundPort(x))), - "TopicMetadataTest-testBasicTopicMetadata", 2000, 0) - topicMetadata.brokers.sortBy(_.id) == foundMetadata.brokers.sortBy(_.id) && - topicMetadata.topicsMetadata.sortBy(_.topic) == foundMetadata.topicsMetadata.sortBy(_.topic) - }, - s"Topic metadata is not correctly updated")) - } - - @Test - def testAliveBrokerListWithNoTopics(): Unit = { - checkMetadata(Seq(server1), 1) - } - - @Test - def testAliveBrokersListWithNoTopicsAfterNewBrokerStartup(): Unit = { - 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) - } - - - @Test - def testAliveBrokersListWithNoTopicsAfterABrokerShutdown(): Unit = { - adHocServers = adHocConfigs.map(p => createServer(p)) - - checkMetadata(adHocServers, numConfigs) - - // Shutdown a broker - adHocServers.last.shutdown() - adHocServers.last.awaitShutdown() - - checkMetadata(adHocServers, numConfigs - 1) - } -} diff --git a/core/src/test/scala/unit/kafka/integration/UncleanLeaderElectionTest.scala b/core/src/test/scala/unit/kafka/integration/UncleanLeaderElectionTest.scala index a15ddb84065e0..040bee3681054 100755 --- a/core/src/test/scala/unit/kafka/integration/UncleanLeaderElectionTest.scala +++ b/core/src/test/scala/unit/kafka/integration/UncleanLeaderElectionTest.scala @@ -21,17 +21,21 @@ import org.apache.kafka.common.config.ConfigException import org.junit.{After, Before, Ignore, Test} import scala.util.Random +import scala.collection.JavaConverters._ import org.apache.log4j.{Level, Logger} import java.util.Properties import java.util.concurrent.ExecutionException -import kafka.consumer.{Consumer, ConsumerConfig} -import kafka.serializer.StringDecoder import kafka.server.{KafkaConfig, KafkaServer} -import kafka.utils.CoreUtils +import kafka.utils.{CoreUtils, TestUtils} import kafka.utils.TestUtils._ import kafka.zk.ZooKeeperTestHarness +import org.apache.kafka.clients.consumer.ConsumerConfig +import org.apache.kafka.common.TopicPartition import org.apache.kafka.common.errors.TimeoutException +import org.apache.kafka.common.network.ListenerName +import org.apache.kafka.common.security.auth.SecurityProtocol +import org.apache.kafka.common.serialization.StringDeserializer import org.junit.Assert._ class UncleanLeaderElectionTest extends ZooKeeperTestHarness { @@ -177,19 +181,19 @@ class UncleanLeaderElectionTest extends ZooKeeperTestHarness { produceMessage(servers, topic, "first") waitUntilMetadataIsPropagated(servers, topic, partitionId) - assertEquals(List("first"), consumeAllMessages(topic)) + assertEquals(List("first"), consumeAllMessages(topic, 1)) // shutdown follower server servers.filter(server => server.config.brokerId == followerId).map(server => shutdownServer(server)) produceMessage(servers, topic, "second") - assertEquals(List("first", "second"), consumeAllMessages(topic)) + assertEquals(List("first", "second"), consumeAllMessages(topic, 2)) //remove any previous unclean election metric - servers.map(server => server.kafkaController.controllerContext.stats.removeMetric("UncleanLeaderElectionsPerSec")) + servers.map(_.kafkaController.controllerContext.stats.removeMetric("UncleanLeaderElectionsPerSec")) // shutdown leader and then restart follower - servers.filter(server => server.config.brokerId == leaderId).map(server => shutdownServer(server)) + servers.filter(_.config.brokerId == leaderId).map(shutdownServer) val followerServer = servers.find(_.config.brokerId == followerId).get followerServer.startup() @@ -200,7 +204,7 @@ class UncleanLeaderElectionTest extends ZooKeeperTestHarness { produceMessage(servers, topic, "third") // second message was lost due to unclean election - assertEquals(List("first", "third"), consumeAllMessages(topic)) + assertEquals(List("first", "third"), consumeAllMessages(topic, 2)) } def verifyUncleanLeaderElectionDisabled(): Unit = { @@ -215,13 +219,13 @@ class UncleanLeaderElectionTest extends ZooKeeperTestHarness { produceMessage(servers, topic, "first") waitUntilMetadataIsPropagated(servers, topic, partitionId) - assertEquals(List("first"), consumeAllMessages(topic)) + assertEquals(List("first"), consumeAllMessages(topic, 1)) // shutdown follower server servers.filter(server => server.config.brokerId == followerId).map(server => shutdownServer(server)) produceMessage(servers, topic, "second") - assertEquals(List("first", "second"), consumeAllMessages(topic)) + assertEquals(List("first", "second"), consumeAllMessages(topic, 2)) //remove any previous unclean election metric servers.map(server => server.kafkaController.controllerContext.stats.removeMetric("UncleanLeaderElectionsPerSec")) @@ -243,7 +247,7 @@ class UncleanLeaderElectionTest extends ZooKeeperTestHarness { case e: ExecutionException if e.getCause.isInstanceOf[TimeoutException] => // expected } - assertEquals(List.empty[String], consumeAllMessages(topic)) + assertEquals(List.empty[String], consumeAllMessages(topic, 0)) // restart leader temporarily to send a successfully replicated message servers.filter(server => server.config.brokerId == leaderId).map(server => server.startup()) @@ -257,7 +261,7 @@ class UncleanLeaderElectionTest extends ZooKeeperTestHarness { waitUntilLeaderIsElectedOrChanged(zkClient, topic, partitionId, newLeaderOpt = Some(followerId)) // verify messages can be consumed from ISR follower that was just promoted to leader - assertEquals(List("first", "second", "third"), consumeAllMessages(topic)) + assertEquals(List("first", "second", "third"), consumeAllMessages(topic, 3)) } private def shutdownServer(server: KafkaServer) = { @@ -265,16 +269,18 @@ class UncleanLeaderElectionTest extends ZooKeeperTestHarness { server.awaitShutdown() } - private def consumeAllMessages(topic: String) : List[String] = { - // use a fresh consumer group every time so that we don't need to mess with disabling auto-commit or - // resetting the ZK offset - val consumerProps = createConsumerProperties(zkConnect, "group" + random.nextLong, "id", 1000) - val consumerConnector = Consumer.create(new ConsumerConfig(consumerProps)) - val messageStream = consumerConnector.createMessageStreams(Map(topic -> 1), new StringDecoder(), new StringDecoder()) - - val messages = getMessages(messageStream) - consumerConnector.shutdown - - messages + private def consumeAllMessages(topic: String, numMessages: Int): Seq[String] = { + val brokerList = TestUtils.bootstrapServers(servers, ListenerName.forSecurityProtocol(SecurityProtocol.PLAINTEXT)) + val props = new Properties + // Don't rely on coordinator as it may be down when this method is called + props.put(ConsumerConfig.ENABLE_AUTO_COMMIT_CONFIG, "false") + val consumer = TestUtils.createConsumer(brokerList, "group" + random.nextLong, + securityProtocol = SecurityProtocol.PLAINTEXT, valueDeserializer = new StringDeserializer, props = Some(props)) + try { + val tp = new TopicPartition(topic, partitionId) + consumer.assign(Seq(tp).asJava) + consumer.seek(tp, 0) + TestUtils.consumeRecords(consumer, numMessages).map(_.value) + } finally consumer.close() } } diff --git a/core/src/test/scala/unit/kafka/javaapi/consumer/ZookeeperConsumerConnectorTest.scala b/core/src/test/scala/unit/kafka/javaapi/consumer/ZookeeperConsumerConnectorTest.scala deleted file mode 100644 index 695b1b692a47e..0000000000000 --- a/core/src/test/scala/unit/kafka/javaapi/consumer/ZookeeperConsumerConnectorTest.scala +++ /dev/null @@ -1,106 +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.javaapi.consumer - -import java.util.Properties - -import kafka.serializer._ -import kafka.server._ -import kafka.integration.KafkaServerTestHarness -import kafka.utils.{Logging, TestUtils} -import kafka.consumer.{ConsumerConfig, KafkaStream} -import kafka.common.MessageStreamsExistException -import org.apache.kafka.clients.producer.ProducerRecord -import org.apache.kafka.common.serialization.{IntegerSerializer, StringSerializer} -import org.junit.Test - -import scala.collection.JavaConverters._ - -import org.apache.log4j.{Level, Logger} -import org.junit.Assert._ - -@deprecated("This test has been deprecated and it will be removed in a future release", "0.10.0.0") -class ZookeeperConsumerConnectorTest 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).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) - - // create the topic - createTopic(topic, numParts, 1) - - // send some messages to each broker - val sentMessages1 = sendMessages(servers, nMessages, "batch1") - - // create a consumer - val consumerConfig1 = new ConsumerConfig(TestUtils.createConsumerProperties(zkConnect, group, consumer1)) - val zkConsumerConnector1 = new ZookeeperConsumerConnector(consumerConfig1, true) - val topicMessageStreams1 = zkConsumerConnector1.createMessageStreams(Map[String, Integer](topic -> numNodes*numParts/2).asJava, new StringDecoder(), new StringDecoder()) - - val receivedMessages1 = getMessages(nMessages*2, topicMessageStreams1) - assertEquals(sentMessages1.sorted, receivedMessages1.sorted) - - // call createMesssageStreams twice should throw MessageStreamsExistException - try { - zkConsumerConnector1.createMessageStreams(Map[String, Integer](topic -> numNodes*numParts/2).asJava, new StringDecoder(), new StringDecoder()) - fail("Should fail with MessageStreamsExistException") - } catch { - case _: MessageStreamsExistException => // expected - } - zkConsumerConnector1.shutdown - info("all consumer connectors stopped") - requestHandlerLogger.setLevel(Level.ERROR) - } - - def sendMessages(servers: Seq[KafkaServer], - messagesPerNode: Int, - header: String): List[String] = { - var messages: List[String] = Nil - val producer = TestUtils.createNewProducer[Integer, String](TestUtils.getBrokerListStrFromServers(servers), - keySerializer = new IntegerSerializer, valueSerializer = new StringSerializer) - for (server <- servers) { - for (partition <- 0 until numParts) { - val ms = (0 until messagesPerNode).map(x => header + server.config.brokerId + "-" + partition + "-" + x) - messages ++= ms - ms.map(new ProducerRecord[Integer, String](topic, partition, partition, _)).map(producer.send).foreach(_.get) - } - } - producer.close() - messages - } - - def getMessages(nMessagesPerThread: Int, - jTopicMessageStreams: java.util.Map[String, java.util.List[KafkaStream[String, String]]]): List[String] = { - val topicMessageStreams = jTopicMessageStreams.asScala.mapValues(_.asScala.toList) - TestUtils.getMessages(topicMessageStreams, nMessagesPerThread) - } -} diff --git a/core/src/test/scala/unit/kafka/javaapi/message/BaseMessageSetTestCases.scala b/core/src/test/scala/unit/kafka/javaapi/message/BaseMessageSetTestCases.scala deleted file mode 100644 index 199bbbdf4ca69..0000000000000 --- a/core/src/test/scala/unit/kafka/javaapi/message/BaseMessageSetTestCases.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.javaapi.message - -import org.junit.Assert._ -import org.scalatest.junit.JUnitSuite -import org.junit.Test -import kafka.message.{CompressionCodec, DefaultCompressionCodec, Message, NoCompressionCodec} -import org.apache.kafka.test.TestUtils - -import scala.collection.JavaConverters._ - -trait BaseMessageSetTestCases extends JUnitSuite { - - val messages = Array(new Message("abcd".getBytes()), new Message("efgh".getBytes())) - def createMessageSet(messages: Seq[Message], compressed: CompressionCodec = NoCompressionCodec): MessageSet - - @Test - def testWrittenEqualsRead(): Unit = { - val messageSet = createMessageSet(messages) - assertEquals(messages.toSeq, messageSet.asScala.map(m => m.message)) - } - - @Test - def testIteratorIsConsistent() { - val m = createMessageSet(messages) - // two iterators over the same set should give the same results - TestUtils.checkEquals(m, m) - } - - @Test - def testIteratorIsConsistentWithCompression() { - val m = createMessageSet(messages, DefaultCompressionCodec) - // two iterators over the same set should give the same results - TestUtils.checkEquals(m, m) - } - - @Test - def testSizeInBytes() { - assertEquals("Empty message set should have 0 bytes.", - 0, - createMessageSet(Array[Message]()).sizeInBytes) - assertEquals("Predicted size should equal actual size.", - kafka.message.MessageSet.messageSetSize(messages), - createMessageSet(messages).sizeInBytes) - } - - @Test - def testSizeInBytesWithCompression () { - assertEquals("Empty message set should have 0 bytes.", - 0, // overhead of the GZIP output stream - createMessageSet(Array[Message](), DefaultCompressionCodec).sizeInBytes) - } -} diff --git a/core/src/test/scala/unit/kafka/javaapi/message/ByteBufferMessageSetTest.scala b/core/src/test/scala/unit/kafka/javaapi/message/ByteBufferMessageSetTest.scala deleted file mode 100644 index fbdb000a1d8c4..0000000000000 --- a/core/src/test/scala/unit/kafka/javaapi/message/ByteBufferMessageSetTest.scala +++ /dev/null @@ -1,46 +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.javaapi.message - -import org.junit.Assert._ -import org.junit.Test -import kafka.message.{DefaultCompressionCodec, CompressionCodec, NoCompressionCodec, Message} - -class ByteBufferMessageSetTest extends kafka.javaapi.message.BaseMessageSetTestCases { - - override def createMessageSet(messages: Seq[Message], compressed: CompressionCodec = NoCompressionCodec): ByteBufferMessageSet = - new ByteBufferMessageSet(new kafka.message.ByteBufferMessageSet(compressed, messages: _*).buffer) - - val msgSeq: Seq[Message] = Seq(new Message("hello".getBytes()), new Message("there".getBytes())) - - @Test - def testEquals() { - val messageList = createMessageSet(msgSeq, NoCompressionCodec) - val moreMessages = createMessageSet(msgSeq, NoCompressionCodec) - assertEquals(messageList, moreMessages) - assertTrue(messageList.equals(moreMessages)) - } - - @Test - def testEqualsWithCompression () { - val messageList = createMessageSet(msgSeq, DefaultCompressionCodec) - val moreMessages = createMessageSet(msgSeq, DefaultCompressionCodec) - assertEquals(messageList, moreMessages) - assertTrue(messageList.equals(moreMessages)) - } -} diff --git a/core/src/test/scala/unit/kafka/log/LogCleanerTest.scala b/core/src/test/scala/unit/kafka/log/LogCleanerTest.scala index 537c561b3872d..0dcfeec9671f5 100755 --- a/core/src/test/scala/unit/kafka/log/LogCleanerTest.scala +++ b/core/src/test/scala/unit/kafka/log/LogCleanerTest.scala @@ -21,6 +21,7 @@ import java.io.{File, RandomAccessFile} import java.nio._ import java.nio.file.Paths import java.util.Properties +import java.util.concurrent.{CountDownLatch, TimeUnit} import kafka.common._ import kafka.server.{BrokerTopicStats, LogDirFailureChannel} @@ -71,11 +72,11 @@ class LogCleanerTest extends JUnitSuite { // append messages to the log until we have four segments while(log.numberOfSegments < 4) log.appendAsLeader(record(log.logEndOffset.toInt, log.logEndOffset.toInt), leaderEpoch = 0) - val keysFound = keysInLog(log) + val keysFound = LogTest.keysInLog(log) assertEquals(0L until log.logEndOffset, keysFound) // pretend we have the following keys - val keys = immutable.ListSet(1, 3, 5, 7, 9) + val keys = immutable.ListSet(1L, 3L, 5L, 7L, 9L) val map = new FakeOffsetMap(Int.MaxValue) keys.foreach(k => map.put(key(k), Long.MaxValue)) @@ -84,11 +85,79 @@ class LogCleanerTest extends JUnitSuite { val stats = new CleanerStats() val expectedBytesRead = segments.map(_.size).sum cleaner.cleanSegments(log, segments, map, 0L, stats) - val shouldRemain = keysInLog(log).filter(!keys.contains(_)) - assertEquals(shouldRemain, keysInLog(log)) + val shouldRemain = LogTest.keysInLog(log).filter(!keys.contains(_)) + assertEquals(shouldRemain, LogTest.keysInLog(log)) assertEquals(expectedBytesRead, stats.bytesRead) } + @Test + def testCleanSegmentsWithConcurrentSegmentDeletion(): Unit = { + val deleteStartLatch = new CountDownLatch(1) + val deleteCompleteLatch = new CountDownLatch(1) + + // Construct a log instance. The replaceSegments() method of the log instance is overridden so that + // it waits for another thread to execute deleteOldSegments() + val logProps = new Properties() + logProps.put(LogConfig.SegmentBytesProp, 1024 : java.lang.Integer) + logProps.put(LogConfig.CleanupPolicyProp, LogConfig.Compact + "," + LogConfig.Delete) + val topicPartition = Log.parseTopicPartitionName(dir) + val producerStateManager = new ProducerStateManager(topicPartition, dir) + val log = new Log(dir, + config = LogConfig.fromProps(logConfig.originals, logProps), + logStartOffset = 0L, + recoveryPoint = 0L, + scheduler = time.scheduler, + brokerTopicStats = new BrokerTopicStats, time, + maxProducerIdExpirationMs = 60 * 60 * 1000, + producerIdExpirationCheckIntervalMs = LogManager.ProducerIdExpirationCheckIntervalMs, + topicPartition = topicPartition, + producerStateManager = producerStateManager, + logDirFailureChannel = new LogDirFailureChannel(10)) { + override def replaceSegments(newSegments: Seq[LogSegment], oldSegments: Seq[LogSegment], isRecoveredSwapFile: Boolean = false): Unit = { + deleteStartLatch.countDown() + if (!deleteCompleteLatch.await(5000, TimeUnit.MILLISECONDS)) { + throw new IllegalStateException("Log segment deletion timed out") + } + super.replaceSegments(newSegments, oldSegments, isRecoveredSwapFile) + } + } + + // Start a thread which execute log.deleteOldSegments() right before replaceSegments() is executed + val t = new Thread() { + override def run(): Unit = { + deleteStartLatch.await(5000, TimeUnit.MILLISECONDS) + log.maybeIncrementLogStartOffset(log.activeSegment.baseOffset) + log.onHighWatermarkIncremented(log.activeSegment.baseOffset) + log.deleteOldSegments() + deleteCompleteLatch.countDown() + } + } + t.start() + + // Append records so that segment number increase to 3 + while (log.numberOfSegments < 3) { + log.appendAsLeader(record(key = 0, log.logEndOffset.toInt), leaderEpoch = 0) + log.roll() + } + assertEquals(3, log.numberOfSegments) + + // Remember reference to the first log and determine its file name expected for async deletion + val firstLogFile = log.logSegments.head.log + val expectedFileName = CoreUtils.replaceSuffix(firstLogFile.file.getPath, "", Log.DeletedFileSuffix) + + // Clean the log. This should trigger replaceSegments() and deleteOldSegments(); + val offsetMap = new FakeOffsetMap(Int.MaxValue) + val cleaner = makeCleaner(Int.MaxValue) + val segments = log.logSegments(0, log.activeSegment.baseOffset).toSeq + val stats = new CleanerStats() + cleaner.buildOffsetMap(log, 0, log.activeSegment.baseOffset, offsetMap, stats) + cleaner.cleanSegments(log, segments, offsetMap, 0L, stats) + + // Validate based on the file name that log segment file is renamed exactly once for async deletion + assertEquals(expectedFileName, firstLogFile.file().getPath) + assertEquals(2, log.numberOfSegments) + } + @Test def testSizeTrimmedForPreallocatedAndCompactedTopic(): Unit = { val originalMaxFileSize = 1024; @@ -135,7 +204,7 @@ class LogCleanerTest extends JUnitSuite { cleaner.clean(LogToClean(new TopicPartition("test", 0), log, 0L, log.activeSegment.baseOffset)) assertEquals(List(2, 5, 7), lastOffsetsPerBatchInLog(log)) assertEquals(Map(pid1 -> 2, pid2 -> 2, pid3 -> 1), lastSequencesInLog(log)) - assertEquals(List(2, 3, 1, 4), keysInLog(log)) + assertEquals(List(2, 3, 1, 4), LogTest.keysInLog(log)) assertEquals(List(1, 3, 6, 7), offsetsInLog(log)) // we have to reload the log to validate that the cleaner maintained sequence numbers correctly @@ -167,7 +236,7 @@ class LogCleanerTest extends JUnitSuite { cleaner.clean(LogToClean(new TopicPartition("test", 0), log, 0L, log.activeSegment.baseOffset)) assertEquals(Map(pid1 -> 2, pid2 -> 2, pid3 -> 1, pid4 -> 0), lastSequencesInLog(log)) assertEquals(List(2, 5, 7, 8), lastOffsetsPerBatchInLog(log)) - assertEquals(List(3, 1, 4, 2), keysInLog(log)) + assertEquals(List(3, 1, 4, 2), LogTest.keysInLog(log)) assertEquals(List(3, 6, 7, 8), offsetsInLog(log)) reloadLog() @@ -204,7 +273,7 @@ class LogCleanerTest extends JUnitSuite { log.roll() cleaner.clean(LogToClean(new TopicPartition("test", 0), log, 0L, log.activeSegment.baseOffset)) - assertEquals(List(3, 2), keysInLog(log)) + assertEquals(List(3, 2), LogTest.keysInLog(log)) assertEquals(List(3, 6, 7, 8, 9), offsetsInLog(log)) // ensure the transaction index is still correct @@ -244,7 +313,7 @@ class LogCleanerTest extends JUnitSuite { // we have only cleaned the records in the first segment val dirtyOffset = cleaner.clean(LogToClean(new TopicPartition("test", 0), log, 0L, log.activeSegment.baseOffset))._1 - assertEquals(List(2, 3, 5, 6, 6, 7, 7, 8, 8, 9, 9, 10), keysInLog(log)) + assertEquals(List(2, 3, 5, 6, 6, 7, 7, 8, 8, 9, 9, 10), LogTest.keysInLog(log)) log.roll() @@ -254,7 +323,7 @@ class LogCleanerTest extends JUnitSuite { // finally only the keys from pid3 should remain cleaner.clean(LogToClean(new TopicPartition("test", 0), log, dirtyOffset, log.activeSegment.baseOffset)) - assertEquals(List(2, 3, 6, 7, 8, 9, 11, 12), keysInLog(log)) + assertEquals(List(2, 3, 6, 7, 8, 9, 11, 12), LogTest.keysInLog(log)) } @Test @@ -278,7 +347,7 @@ class LogCleanerTest extends JUnitSuite { // cannot remove the marker in this pass because there are still valid records var dirtyOffset = cleaner.doClean(LogToClean(tp, log, 0L, 100L), deleteHorizonMs = Long.MaxValue)._1 - assertEquals(List(1, 3, 2), keysInLog(log)) + assertEquals(List(1, 3, 2), LogTest.keysInLog(log)) assertEquals(List(0, 2, 3, 4, 5), offsetsInLog(log)) appendProducer(Seq(1, 3)) @@ -287,17 +356,17 @@ class LogCleanerTest extends JUnitSuite { // the first cleaning preserves the commit marker (at offset 3) since there were still records for the transaction dirtyOffset = cleaner.doClean(LogToClean(tp, log, dirtyOffset, 100L), deleteHorizonMs = Long.MaxValue)._1 - assertEquals(List(2, 1, 3), keysInLog(log)) + assertEquals(List(2, 1, 3), LogTest.keysInLog(log)) assertEquals(List(3, 4, 5, 6, 7, 8), offsetsInLog(log)) // delete horizon forced to 0 to verify marker is not removed early dirtyOffset = cleaner.doClean(LogToClean(tp, log, dirtyOffset, 100L), deleteHorizonMs = 0L)._1 - assertEquals(List(2, 1, 3), keysInLog(log)) + assertEquals(List(2, 1, 3), LogTest.keysInLog(log)) assertEquals(List(3, 4, 5, 6, 7, 8), offsetsInLog(log)) // clean again with large delete horizon and verify the marker is removed dirtyOffset = cleaner.doClean(LogToClean(tp, log, dirtyOffset, 100L), deleteHorizonMs = Long.MaxValue)._1 - assertEquals(List(2, 1, 3), keysInLog(log)) + assertEquals(List(2, 1, 3), LogTest.keysInLog(log)) assertEquals(List(4, 5, 6, 7, 8), offsetsInLog(log)) } @@ -326,11 +395,11 @@ class LogCleanerTest extends JUnitSuite { log.roll() cleaner.doClean(LogToClean(tp, log, 0L, 100L), deleteHorizonMs = Long.MaxValue) - assertEquals(List(2), keysInLog(log)) + assertEquals(List(2), LogTest.keysInLog(log)) assertEquals(List(1, 3, 4), offsetsInLog(log)) cleaner.doClean(LogToClean(tp, log, 0L, 100L), deleteHorizonMs = Long.MaxValue) - assertEquals(List(2), keysInLog(log)) + assertEquals(List(2), LogTest.keysInLog(log)) assertEquals(List(3, 4), offsetsInLog(log)) } @@ -343,42 +412,88 @@ class LogCleanerTest extends JUnitSuite { val log = makeLog(config = LogConfig.fromProps(logConfig.originals, logProps)) val producerEpoch = 0.toShort - val producerId = 1L - val appendProducer = appendTransactionalAsLeader(log, producerId, producerEpoch) + val producer1 = appendTransactionalAsLeader(log, 1L, producerEpoch) + val producer2 = appendTransactionalAsLeader(log, 2L, producerEpoch) - appendProducer(Seq(2, 3)) // batch last offset is 1 - log.appendAsLeader(commitMarker(producerId, producerEpoch), leaderEpoch = 0, isFromClient = false) + // [{Producer1: 2, 3}] + producer1(Seq(2, 3)) // offsets 0, 1 log.roll() - log.appendAsLeader(record(2, 2), leaderEpoch = 0) - log.appendAsLeader(record(3, 3), leaderEpoch = 0) + // [{Producer1: 2, 3}], [{Producer2: 2, 3}, {Producer2: Commit}] + producer2(Seq(2, 3)) // offsets 2, 3 + log.appendAsLeader(commitMarker(2L, producerEpoch), leaderEpoch = 0, isFromClient = false) // offset 4 + log.roll() + + // [{Producer1: 2, 3}], [{Producer2: 2, 3}, {Producer2: Commit}], [{2}, {3}, {Producer1: Commit}] + // {0, 1}, {2, 3}, {4}, {5}, {6}, {7} ==> Offsets + log.appendAsLeader(record(2, 2), leaderEpoch = 0) // offset 5 + log.appendAsLeader(record(3, 3), leaderEpoch = 0) // offset 6 + log.appendAsLeader(commitMarker(1L, producerEpoch), leaderEpoch = 0, isFromClient = false) // offset 7 log.roll() // first time through the records are removed + // Expected State: [{Producer1: EmptyBatch}, {Producer2: EmptyBatch}, {Producer2: Commit}, {2}, {3}] var dirtyOffset = cleaner.doClean(LogToClean(tp, log, 0L, 100L), deleteHorizonMs = Long.MaxValue)._1 - assertEquals(List(2, 3), keysInLog(log)) - assertEquals(List(2, 3, 4), offsetsInLog(log)) // commit marker is retained - assertEquals(List(1, 2, 3, 4), lastOffsetsPerBatchInLog(log)) // empty batch is retained + assertEquals(List(2, 3), LogTest.keysInLog(log)) + assertEquals(List(4, 5, 6), offsetsInLog(log)) + assertEquals(List(1, 3, 4, 5, 6), lastOffsetsPerBatchInLog(log)) // the empty batch remains if cleaned again because it still holds the last sequence + // Expected State: [{Producer1: EmptyBatch}, {Producer2: EmptyBatch}, {Producer2: Commit}, {2}, {3}] dirtyOffset = cleaner.doClean(LogToClean(tp, log, dirtyOffset, 100L), deleteHorizonMs = Long.MaxValue)._1 - assertEquals(List(2, 3), keysInLog(log)) - assertEquals(List(2, 3, 4), offsetsInLog(log)) // commit marker is still retained - assertEquals(List(1, 2, 3, 4), lastOffsetsPerBatchInLog(log)) // empty batch is retained + assertEquals(List(2, 3), LogTest.keysInLog(log)) + assertEquals(List(4, 5, 6), offsetsInLog(log)) + assertEquals(List(1, 3, 4, 5, 6), lastOffsetsPerBatchInLog(log)) // append a new record from the producer to allow cleaning of the empty batch - appendProducer(Seq(1)) + // [{Producer1: EmptyBatch}, {Producer2: EmptyBatch}, {Producer2: Commit}, {2}, {3}] [{Producer2: 1}, {Producer2: Commit}] + // {1}, {3}, {4}, {5}, {6}, {8}, {9} ==> Offsets + producer2(Seq(1)) // offset 8 + log.appendAsLeader(commitMarker(2L, producerEpoch), leaderEpoch = 0, isFromClient = false) // offset 9 log.roll() + // Expected State: [{Producer1: EmptyBatch}, {Producer2: Commit}, {2}, {3}, {Producer2: 1}, {Producer2: Commit}] dirtyOffset = cleaner.doClean(LogToClean(tp, log, dirtyOffset, 100L), deleteHorizonMs = Long.MaxValue)._1 - assertEquals(List(2, 3, 1), keysInLog(log)) - assertEquals(List(2, 3, 4, 5), offsetsInLog(log)) // commit marker is still retained - assertEquals(List(2, 3, 4, 5), lastOffsetsPerBatchInLog(log)) // empty batch should be gone + assertEquals(List(2, 3, 1), LogTest.keysInLog(log)) + assertEquals(List(4, 5, 6, 8, 9), offsetsInLog(log)) + assertEquals(List(1, 4, 5, 6, 8, 9), lastOffsetsPerBatchInLog(log)) + + // Expected State: [{Producer1: EmptyBatch}, {2}, {3}, {Producer2: 1}, {Producer2: Commit}] + dirtyOffset = cleaner.doClean(LogToClean(tp, log, dirtyOffset, 100L), deleteHorizonMs = Long.MaxValue)._1 + assertEquals(List(2, 3, 1), LogTest.keysInLog(log)) + assertEquals(List(5, 6, 8, 9), offsetsInLog(log)) + assertEquals(List(1, 5, 6, 8, 9), lastOffsetsPerBatchInLog(log)) + } + + @Test + def testCleanEmptyControlBatch(): Unit = { + val tp = new TopicPartition("test", 0) + val cleaner = makeCleaner(Int.MaxValue) + val logProps = new Properties() + logProps.put(LogConfig.SegmentBytesProp, 256: java.lang.Integer) + val log = makeLog(config = LogConfig.fromProps(logConfig.originals, logProps)) + + val producerEpoch = 0.toShort + + // [{Producer1: Commit}, {2}, {3}] + log.appendAsLeader(commitMarker(1L, producerEpoch), leaderEpoch = 0, isFromClient = false) // offset 7 + log.appendAsLeader(record(2, 2), leaderEpoch = 0) // offset 2 + log.appendAsLeader(record(3, 3), leaderEpoch = 0) // offset 3 + log.roll() + // first time through the control batch is retained as an empty batch + // Expected State: [{Producer1: EmptyBatch}], [{2}, {3}] + var dirtyOffset = cleaner.doClean(LogToClean(tp, log, 0L, 100L), deleteHorizonMs = Long.MaxValue)._1 + assertEquals(List(2, 3), LogTest.keysInLog(log)) + assertEquals(List(1, 2), offsetsInLog(log)) + assertEquals(List(0, 1, 2), lastOffsetsPerBatchInLog(log)) + + // the empty control batch does not cause an exception when cleaned + // Expected State: [{Producer1: EmptyBatch}], [{2}, {3}] dirtyOffset = cleaner.doClean(LogToClean(tp, log, dirtyOffset, 100L), deleteHorizonMs = Long.MaxValue)._1 - assertEquals(List(2, 3, 1), keysInLog(log)) - assertEquals(List(3, 4, 5), offsetsInLog(log)) // commit marker is gone - assertEquals(List(3, 4, 5), lastOffsetsPerBatchInLog(log)) // empty batch is gone + assertEquals(List(2, 3), LogTest.keysInLog(log)) + assertEquals(List(1, 2), offsetsInLog(log)) + assertEquals(List(0, 1, 2), lastOffsetsPerBatchInLog(log)) } @Test @@ -402,15 +517,54 @@ class LogCleanerTest extends JUnitSuite { // delete horizon set to 0 to verify marker is not removed early val dirtyOffset = cleaner.doClean(LogToClean(tp, log, 0L, 100L), deleteHorizonMs = 0L)._1 - assertEquals(List(3), keysInLog(log)) + assertEquals(List(3), LogTest.keysInLog(log)) assertEquals(List(3, 4, 5), offsetsInLog(log)) // clean again with large delete horizon and verify the marker is removed cleaner.doClean(LogToClean(tp, log, dirtyOffset, 100L), deleteHorizonMs = Long.MaxValue) - assertEquals(List(3), keysInLog(log)) + assertEquals(List(3), LogTest.keysInLog(log)) assertEquals(List(4, 5), offsetsInLog(log)) } + @Test + def testEmptyBatchRemovalWithSequenceReuse(): Unit = { + // The group coordinator always writes batches beginning with sequence number 0. This test + // ensures that we still remove old empty batches and transaction markers under this expectation. + + val producerEpoch = 0.toShort + val producerId = 1L + val tp = new TopicPartition("test", 0) + val cleaner = makeCleaner(Int.MaxValue) + val logProps = new Properties() + logProps.put(LogConfig.SegmentBytesProp, 2048: java.lang.Integer) + val log = makeLog(config = LogConfig.fromProps(logConfig.originals, logProps)) + + val appendFirstTransaction = appendTransactionalAsLeader(log, producerId, producerEpoch, isFromClient = false) + appendFirstTransaction(Seq(1)) + log.appendAsLeader(commitMarker(producerId, producerEpoch), leaderEpoch = 0, isFromClient = false) + + val appendSecondTransaction = appendTransactionalAsLeader(log, producerId, producerEpoch, isFromClient = false) + appendSecondTransaction(Seq(2)) + log.appendAsLeader(commitMarker(producerId, producerEpoch), leaderEpoch = 0, isFromClient = false) + + log.appendAsLeader(record(1, 1), leaderEpoch = 0) + log.appendAsLeader(record(2, 1), leaderEpoch = 0) + + // Roll the log to ensure that the data is cleanable. + log.roll() + + // Both transactional batches will be cleaned. The last one will remain in the log + // as an empty batch in order to preserve the producer sequence number and epoch + cleaner.doClean(LogToClean(tp, log, 0L, 100L), deleteHorizonMs = Long.MaxValue) + assertEquals(List(1, 3, 4, 5), offsetsInLog(log)) + assertEquals(List(1, 2, 3, 4, 5), lastOffsetsPerBatchInLog(log)) + + // On the second round of cleaning, the marker from the first transaction should be removed. + cleaner.doClean(LogToClean(tp, log, 0L, 100L), deleteHorizonMs = Long.MaxValue) + assertEquals(List(3, 4, 5), offsetsInLog(log)) + assertEquals(List(2, 3, 4, 5), lastOffsetsPerBatchInLog(log)) + } + @Test def testAbortMarkerRetentionWithEmptyBatch(): Unit = { val tp = new TopicPartition("test", 0) @@ -440,14 +594,14 @@ class LogCleanerTest extends JUnitSuite { // first time through the records are removed var dirtyOffset = cleaner.doClean(LogToClean(tp, log, 0L, 100L), deleteHorizonMs = Long.MaxValue)._1 assertAbortedTransactionIndexed() - assertEquals(List(), keysInLog(log)) + assertEquals(List(), LogTest.keysInLog(log)) assertEquals(List(2), offsetsInLog(log)) // abort marker is retained assertEquals(List(1, 2), lastOffsetsPerBatchInLog(log)) // empty batch is retained // the empty batch remains if cleaned again because it still holds the last sequence dirtyOffset = cleaner.doClean(LogToClean(tp, log, dirtyOffset, 100L), deleteHorizonMs = Long.MaxValue)._1 assertAbortedTransactionIndexed() - assertEquals(List(), keysInLog(log)) + assertEquals(List(), LogTest.keysInLog(log)) assertEquals(List(2), offsetsInLog(log)) // abort marker is still retained assertEquals(List(1, 2), lastOffsetsPerBatchInLog(log)) // empty batch is retained @@ -457,12 +611,12 @@ class LogCleanerTest extends JUnitSuite { dirtyOffset = cleaner.doClean(LogToClean(tp, log, dirtyOffset, 100L), deleteHorizonMs = Long.MaxValue)._1 assertAbortedTransactionIndexed() - assertEquals(List(1), keysInLog(log)) + assertEquals(List(1), LogTest.keysInLog(log)) assertEquals(List(2, 3), offsetsInLog(log)) // abort marker is not yet gone because we read the empty batch assertEquals(List(2, 3), lastOffsetsPerBatchInLog(log)) // but we do not preserve the empty batch dirtyOffset = cleaner.doClean(LogToClean(tp, log, dirtyOffset, 100L), deleteHorizonMs = Long.MaxValue)._1 - assertEquals(List(1), keysInLog(log)) + assertEquals(List(1), LogTest.keysInLog(log)) assertEquals(List(3), offsetsInLog(log)) // abort marker is gone assertEquals(List(3), lastOffsetsPerBatchInLog(log)) @@ -486,19 +640,19 @@ class LogCleanerTest extends JUnitSuite { while(log.numberOfSegments < 2) log.appendAsLeader(record(log.logEndOffset.toInt, Array.fill(largeMessageSize)(0: Byte)), leaderEpoch = 0) - val keysFound = keysInLog(log) + val keysFound = LogTest.keysInLog(log) assertEquals(0L until log.logEndOffset, keysFound) // pretend we have the following keys - val keys = immutable.ListSet(1, 3, 5, 7, 9) + val keys = immutable.ListSet(1L, 3L, 5L, 7L, 9L) val map = new FakeOffsetMap(Int.MaxValue) keys.foreach(k => map.put(key(k), Long.MaxValue)) // clean the log val stats = new CleanerStats() cleaner.cleanSegments(log, Seq(log.logSegments.head), map, 0L, stats) - val shouldRemain = keysInLog(log).filter(!keys.contains(_)) - assertEquals(shouldRemain, keysInLog(log)) + val shouldRemain = LogTest.keysInLog(log).filter(!keys.contains(_)) + assertEquals(shouldRemain, LogTest.keysInLog(log)) } /** @@ -510,8 +664,8 @@ class LogCleanerTest extends JUnitSuite { val cleaner = makeCleaner(Int.MaxValue, maxMessageSize=1024) cleaner.cleanSegments(log, Seq(log.logSegments.head), offsetMap, 0L, new CleanerStats) - val shouldRemain = keysInLog(log).filter(k => !offsetMap.map.containsKey(k.toString)) - assertEquals(shouldRemain, keysInLog(log)) + val shouldRemain = LogTest.keysInLog(log).filter(k => !offsetMap.map.containsKey(k.toString)) + assertEquals(shouldRemain, LogTest.keysInLog(log)) } /** @@ -558,7 +712,7 @@ class LogCleanerTest extends JUnitSuite { while(log.numberOfSegments < 2) log.appendAsLeader(record(log.logEndOffset.toInt, Array.fill(largeMessageSize)(0: Byte)), leaderEpoch = 0) - val keysFound = keysInLog(log) + val keysFound = LogTest.keysInLog(log) assertEquals(0L until log.logEndOffset, keysFound) // Decrease the log's max message size @@ -595,7 +749,7 @@ class LogCleanerTest extends JUnitSuite { log.appendAsLeader(record(log.logEndOffset.toInt, log.logEndOffset.toInt), leaderEpoch = 0) cleaner.clean(LogToClean(new TopicPartition("test", 0), log, 0, log.activeSegment.baseOffset)) - val keys = keysInLog(log).toSet + val keys = LogTest.keysInLog(log).toSet assertTrue("None of the keys we deleted should still exist.", (0 until leo.toInt by 2).forall(!keys.contains(_))) } @@ -647,7 +801,7 @@ class LogCleanerTest extends JUnitSuite { cleaner.clean(LogToClean(new TopicPartition("test", 0), log, 0L, log.activeSegment.baseOffset)) assertEquals(List(1, 3, 4), lastOffsetsPerBatchInLog(log)) assertEquals(Map(1L -> 0, 2L -> 1, 3L -> 0), lastSequencesInLog(log)) - assertEquals(List(0, 1), keysInLog(log)) + assertEquals(List(0, 1), LogTest.keysInLog(log)) assertEquals(List(3, 4), offsetsInLog(log)) } @@ -670,7 +824,7 @@ class LogCleanerTest extends JUnitSuite { cleaner.clean(LogToClean(new TopicPartition("test", 0), log, 0L, log.activeSegment.baseOffset)) assertEquals(List(2, 3), lastOffsetsPerBatchInLog(log)) assertEquals(Map(producerId -> 2), lastSequencesInLog(log)) - assertEquals(List(), keysInLog(log)) + assertEquals(List(), LogTest.keysInLog(log)) assertEquals(List(3), offsetsInLog(log)) // Append a new entry from the producer and verify that the empty batch is cleaned up @@ -680,7 +834,7 @@ class LogCleanerTest extends JUnitSuite { assertEquals(List(3, 5), lastOffsetsPerBatchInLog(log)) assertEquals(Map(producerId -> 4), lastSequencesInLog(log)) - assertEquals(List(1, 5), keysInLog(log)) + assertEquals(List(1, 5), LogTest.keysInLog(log)) assertEquals(List(3, 4, 5), offsetsInLog(log)) } @@ -703,16 +857,16 @@ class LogCleanerTest extends JUnitSuite { // clean the log with only one message removed cleaner.clean(LogToClean(new TopicPartition("test", 0), log, 2, log.activeSegment.baseOffset)) - assertEquals(List(1,0,1,0), keysInLog(log)) + assertEquals(List(1,0,1,0), LogTest.keysInLog(log)) assertEquals(List(1,2,3,4), offsetsInLog(log)) // continue to make progress, even though we can only clean one message at a time cleaner.clean(LogToClean(new TopicPartition("test", 0), log, 3, log.activeSegment.baseOffset)) - assertEquals(List(0,1,0), keysInLog(log)) + assertEquals(List(0,1,0), LogTest.keysInLog(log)) assertEquals(List(2,3,4), offsetsInLog(log)) cleaner.clean(LogToClean(new TopicPartition("test", 0), log, 4, log.activeSegment.baseOffset)) - assertEquals(List(1,0), keysInLog(log)) + assertEquals(List(1,0), LogTest.keysInLog(log)) assertEquals(List(3,4), offsetsInLog(log)) } @@ -835,14 +989,6 @@ class LogCleanerTest extends JUnitSuite { assertEquals("Cleaner should have seen %d invalid messages.", numInvalidMessages, stats.invalidMessagesRead) } - /* extract all the keys from a log */ - def keysInLog(log: Log): Iterable[Int] = { - for (segment <- log.logSegments; - batch <- segment.log.batches.asScala if !batch.isControlBatch; - record <- batch.asScala if record.hasValue && record.hasKey) - yield TestUtils.readString(record.key).toInt - } - def lastOffsetsPerBatchInLog(log: Log): Iterable[Long] = { for (segment <- log.logSegments; batch <- segment.log.batches.asScala) yield batch.lastOffset @@ -880,7 +1026,7 @@ class LogCleanerTest extends JUnitSuite { while(log.numberOfSegments < 4) log.appendAsLeader(record(log.logEndOffset.toInt, log.logEndOffset.toInt), leaderEpoch = 0) - val keys = keysInLog(log) + val keys = LogTest.keysInLog(log) val map = new FakeOffsetMap(Int.MaxValue) keys.foreach(k => map.put(key(k), Long.MaxValue)) intercept[LogCleaningAbortedException] { @@ -1011,7 +1157,7 @@ class LogCleanerTest extends JUnitSuite { log.appendAsFollower(record1) val record2 = messageWithOffset("hello".getBytes, "hello".getBytes, 1) log.appendAsFollower(record2) - log.roll(Int.MaxValue/2) // starting a new log segment at offset Int.MaxValue/2 + log.roll(Some(Int.MaxValue/2)) // starting a new log segment at offset Int.MaxValue/2 val record3 = messageWithOffset("hello".getBytes, "hello".getBytes, Int.MaxValue/2) log.appendAsFollower(record3) val record4 = messageWithOffset("hello".getBytes, "hello".getBytes, Int.MaxValue.toLong + 1) @@ -1065,6 +1211,48 @@ class LogCleanerTest extends JUnitSuite { checkRange(map, segments(3).baseOffset.toInt, log.logEndOffset.toInt) } + @Test + def testSegmentWithOffsetOverflow(): Unit = { + val cleaner = makeCleaner(Int.MaxValue) + val logProps = new Properties() + logProps.put(LogConfig.IndexIntervalBytesProp, 1: java.lang.Integer) + logProps.put(LogConfig.FileDeleteDelayMsProp, 1000: java.lang.Integer) + val config = LogConfig.fromProps(logConfig.originals, logProps) + + LogTest.initializeLogDirWithOverflowedSegment(dir) + + val log = makeLog(config = config, recoveryPoint = Long.MaxValue) + val segmentWithOverflow = LogTest.firstOverflowSegment(log).getOrElse { + fail("Failed to create log with a segment which has overflowed offsets") + } + + val numSegmentsInitial = log.logSegments.size + val allKeys = LogTest.keysInLog(log).toList + val expectedKeysAfterCleaning = mutable.MutableList[Long]() + + // pretend we want to clean every alternate key + val offsetMap = new FakeOffsetMap(Int.MaxValue) + for (k <- 1 until allKeys.size by 2) { + expectedKeysAfterCleaning += allKeys(k - 1) + offsetMap.put(key(allKeys(k)), Long.MaxValue) + } + + // Try to clean segment with offset overflow. This will trigger log split and the cleaning itself must abort. + assertThrows[LogCleaningAbortedException] { + cleaner.cleanSegments(log, List(segmentWithOverflow), offsetMap, 0L, new CleanerStats()) + } + assertEquals(numSegmentsInitial + 1, log.logSegments.size) + assertEquals(allKeys, LogTest.keysInLog(log)) + assertFalse(LogTest.hasOffsetOverflow(log)) + + // Clean each segment now that split is complete. + for (segmentToClean <- log.logSegments) + cleaner.cleanSegments(log, List(segmentToClean), offsetMap, 0L, new CleanerStats()) + assertEquals(expectedKeysAfterCleaning, LogTest.keysInLog(log)) + assertFalse(LogTest.hasOffsetOverflow(log)) + log.close() + } + /** * Tests recovery if broker crashes at the following stages during the cleaning sequence *
    @@ -1084,28 +1272,14 @@ class LogCleanerTest extends JUnitSuite { 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 + // create a log and append some messages var log = makeLog(config = config) var messageCount = 0 while (log.numberOfSegments < 10) { log.appendAsLeader(record(log.logEndOffset.toInt, log.logEndOffset.toInt), leaderEpoch = 0) messageCount += 1 } - val allKeys = keysInLog(log) + val allKeys = LogTest.keysInLog(log) // pretend we have odd-numbered keys val offsetMap = new FakeOffsetMap(Int.MaxValue) @@ -1116,7 +1290,7 @@ class LogCleanerTest extends JUnitSuite { cleaner.cleanSegments(log, log.logSegments.take(9).toSeq, offsetMap, 0L, new CleanerStats()) // clear scheduler so that async deletes don't run time.scheduler.clear() - var cleanedKeys = keysInLog(log) + var cleanedKeys = LogTest.keysInLog(log) log.close() // 1) Simulate recovery just after .cleaned file is created, before rename to .swap @@ -1131,7 +1305,7 @@ class LogCleanerTest extends JUnitSuite { cleaner.cleanSegments(log, log.logSegments.take(9).toSeq, offsetMap, 0L, new CleanerStats()) // clear scheduler so that async deletes don't run time.scheduler.clear() - cleanedKeys = keysInLog(log) + cleanedKeys = LogTest.keysInLog(log) log.close() // 2) Simulate recovery just after swap file is created, before old segment files are @@ -1152,7 +1326,7 @@ class LogCleanerTest extends JUnitSuite { cleaner.cleanSegments(log, log.logSegments.take(9).toSeq, offsetMap, 0L, new CleanerStats()) // clear scheduler so that async deletes don't run time.scheduler.clear() - cleanedKeys = keysInLog(log) + cleanedKeys = LogTest.keysInLog(log) // 3) Simulate recovery after swap file is created and old segments files are renamed // to .deleted. Clean operation is resumed during recovery. @@ -1169,7 +1343,7 @@ class LogCleanerTest extends JUnitSuite { cleaner.cleanSegments(log, log.logSegments.take(9).toSeq, offsetMap, 0L, new CleanerStats()) // clear scheduler so that async deletes don't run time.scheduler.clear() - cleanedKeys = keysInLog(log) + cleanedKeys = LogTest.keysInLog(log) log.close() // 4) Simulate recovery after swap is complete, but async deletion @@ -1375,7 +1549,7 @@ class LogCleanerTest extends JUnitSuite { for ((key, value) <- seq) yield log.appendAsLeader(record(key, value), leaderEpoch = 0).firstOffset.get } - private def key(id: Int) = ByteBuffer.wrap(id.toString.getBytes) + private def key(id: Long) = ByteBuffer.wrap(id.toString.getBytes) private def record(key: Int, value: Int, producerId: Long = RecordBatch.NO_PRODUCER_ID, @@ -1386,13 +1560,19 @@ class LogCleanerTest extends JUnitSuite { partitionLeaderEpoch, new SimpleRecord(key.toString.getBytes, value.toString.getBytes)) } - private def appendTransactionalAsLeader(log: Log, producerId: Long, producerEpoch: Short): Seq[Int] => LogAppendInfo = { - appendIdempotentAsLeader(log, producerId, producerEpoch, isTransactional = true) + private def appendTransactionalAsLeader(log: Log, + producerId: Long, + producerEpoch: Short, + leaderEpoch: Int = 0, + isFromClient: Boolean = true): Seq[Int] => LogAppendInfo = { + appendIdempotentAsLeader(log, producerId, producerEpoch, isTransactional = true, isFromClient = isFromClient) } private def appendIdempotentAsLeader(log: Log, producerId: Long, producerEpoch: Short, - isTransactional: Boolean = false): Seq[Int] => LogAppendInfo = { + isTransactional: Boolean = false, + leaderEpoch: Int = 0, + isFromClient: Boolean = true): Seq[Int] => LogAppendInfo = { var sequence = 0 keys: Seq[Int] => { val simpleRecords = keys.map { key => @@ -1404,7 +1584,7 @@ class LogCleanerTest extends JUnitSuite { else MemoryRecords.withIdempotentRecords(CompressionType.NONE, producerId, producerEpoch, sequence, simpleRecords: _*) sequence += simpleRecords.size - log.appendAsLeader(records, leaderEpoch = 0) + log.appendAsLeader(records, leaderEpoch, isFromClient) } } @@ -1429,6 +1609,9 @@ class LogCleanerTest extends JUnitSuite { private def tombstoneRecord(key: Int): MemoryRecords = record(key, null) + private def recoverAndCheck(config: LogConfig, expectedKeys: Iterable[Long]): Log = { + LogTest.recoverAndCheck(dir, config, expectedKeys, new BrokerTopicStats(), time, time.scheduler) + } } class FakeOffsetMap(val slots: Int) extends OffsetMap { diff --git a/core/src/test/scala/unit/kafka/log/LogManagerTest.scala b/core/src/test/scala/unit/kafka/log/LogManagerTest.scala index d9efc236780bb..38d6f71c7d0b1 100755 --- a/core/src/test/scala/unit/kafka/log/LogManagerTest.scala +++ b/core/src/test/scala/unit/kafka/log/LogManagerTest.scala @@ -20,10 +20,9 @@ package kafka.log import java.io._ import java.util.Properties -import kafka.common._ import kafka.server.checkpoints.OffsetCheckpointFile import kafka.utils._ -import org.apache.kafka.common.TopicPartition +import org.apache.kafka.common.{KafkaException, TopicPartition} import org.apache.kafka.common.errors.OffsetOutOfRangeException import org.apache.kafka.common.utils.Utils import org.junit.Assert._ @@ -67,6 +66,29 @@ class LogManagerTest { @Test def testCreateLog() { val log = logManager.getOrCreateLog(new TopicPartition(name, 0), logConfig) + assertEquals(1, logManager.liveLogDirs.size) + + val logFile = new File(logDir, name + "-0") + assertTrue(logFile.exists) + log.appendAsLeader(TestUtils.singletonRecords("test".getBytes()), leaderEpoch = 0) + } + + /** + * Test that getOrCreateLog on a non-existent log creates a new log and that we can append to the new log. + * The LogManager is configured with one invalid log directory which should be marked as offline. + */ + @Test + def testCreateLogWithInvalidLogDir() { + // Configure the log dir with the Nul character as the path, which causes dir.getCanonicalPath() to throw an + // IOException. This simulates the scenario where the disk is not properly mounted (which is hard to achieve in + // a unit test) + val dirs = Seq(logDir, new File("\u0000")) + + logManager.shutdown() + logManager = createLogManager(dirs) + logManager.startup() + + val log = logManager.getOrCreateLog(new TopicPartition(name, 0), logConfig, isNew = true) val logFile = new File(logDir, name + "-0") assertTrue(logFile.exists) log.appendAsLeader(TestUtils.singletonRecords("test".getBytes()), leaderEpoch = 0) diff --git a/core/src/test/scala/unit/kafka/log/LogSegmentTest.scala b/core/src/test/scala/unit/kafka/log/LogSegmentTest.scala index 79d6ea2326703..8976c68041945 100644 --- a/core/src/test/scala/unit/kafka/log/LogSegmentTest.scala +++ b/core/src/test/scala/unit/kafka/log/LogSegmentTest.scala @@ -517,4 +517,34 @@ class LogSegmentTest { assertEquals(1, log.records.batches.asScala.size) } + @Test + def testAppendFromFile(): Unit = { + def records(offset: Long, size: Int): MemoryRecords = + MemoryRecords.withRecords(RecordBatch.MAGIC_VALUE_V2, offset, CompressionType.NONE, TimestampType.CREATE_TIME, + new SimpleRecord(new Array[Byte](size))) + + // create a log file in a separate directory to avoid conflicting with created segments + val tempDir = TestUtils.tempDir() + val fileRecords = FileRecords.open(Log.logFile(tempDir, 0)) + + // Simulate a scenario where we have a single log with an offset range exceeding Int.MaxValue + fileRecords.append(records(0, 1024)) + fileRecords.append(records(500, 1024 * 1024 + 1)) + val sizeBeforeOverflow = fileRecords.sizeInBytes() + fileRecords.append(records(Int.MaxValue + 5L, 1024)) + val sizeAfterOverflow = fileRecords.sizeInBytes() + + val segment = createSegment(0) + val bytesAppended = segment.appendFromFile(fileRecords, 0) + assertEquals(sizeBeforeOverflow, bytesAppended) + assertEquals(sizeBeforeOverflow, segment.size) + + val overflowSegment = createSegment(Int.MaxValue) + val overflowBytesAppended = overflowSegment.appendFromFile(fileRecords, sizeBeforeOverflow) + assertEquals(sizeAfterOverflow - sizeBeforeOverflow, overflowBytesAppended) + assertEquals(overflowBytesAppended, overflowSegment.size) + + Utils.delete(tempDir) + } + } diff --git a/core/src/test/scala/unit/kafka/log/LogTest.scala b/core/src/test/scala/unit/kafka/log/LogTest.scala index bf74a3e3995ac..c89a3f74119de 100755 --- a/core/src/test/scala/unit/kafka/log/LogTest.scala +++ b/core/src/test/scala/unit/kafka/log/LogTest.scala @@ -19,36 +19,41 @@ package kafka.log import java.io._ import java.nio.ByteBuffer -import java.nio.file.Files +import java.nio.file.{Files, Paths} import java.util.Properties -import org.apache.kafka.common.errors._ -import kafka.common.KafkaException +import kafka.api.{ApiVersion, KAFKA_0_11_0_IV0} +import kafka.common.{OffsetsOutOfOrderException, UnexpectedAppendOffsetException} import kafka.log.Log.DeleteDirSuffix -import org.junit.Assert._ -import org.junit.{After, Before, Test} -import kafka.utils._ +import kafka.server.checkpoints.{LeaderEpochCheckpointFile, LeaderEpochFile} +import kafka.server.epoch.{EpochEntry, LeaderEpochFileCache} import kafka.server.{BrokerTopicStats, FetchDataInfo, KafkaConfig, LogDirFailureChannel} -import kafka.server.epoch.{EpochEntry, LeaderEpochCache, LeaderEpochFileCache} -import org.apache.kafka.common.TopicPartition +import kafka.utils._ +import org.apache.kafka.common.{KafkaException, TopicPartition} +import org.apache.kafka.common.errors._ import org.apache.kafka.common.record.MemoryRecords.RecordFilter import org.apache.kafka.common.record.MemoryRecords.RecordFilter.BatchRetention import org.apache.kafka.common.record._ +import org.apache.kafka.common.requests.EpochEndOffset.{UNDEFINED_EPOCH, UNDEFINED_EPOCH_OFFSET} import org.apache.kafka.common.requests.FetchResponse.AbortedTransaction import org.apache.kafka.common.requests.IsolationLevel import org.apache.kafka.common.utils.{Time, Utils} import org.easymock.EasyMock +import org.junit.Assert._ +import org.junit.{After, Before, Test} +import org.scalatest.Assertions +import scala.collection.{Iterable, mutable} import scala.collection.JavaConverters._ -import scala.collection.mutable.{ArrayBuffer, ListBuffer} +import scala.collection.mutable.ListBuffer +import org.scalatest.Assertions.{assertThrows, intercept, withClue} class LogTest { - + var config: KafkaConfig = null + val brokerTopicStats = new BrokerTopicStats val tmpDir = TestUtils.tempDir() val logDir = TestUtils.randomPartitionLogDir(tmpDir) val mockTime = new MockTime() - var config: KafkaConfig = null - val brokerTopicStats = new BrokerTopicStats @Before def setUp() { @@ -93,7 +98,7 @@ class LogTest { @Test def testTimeBasedLogRoll() { def createRecords = TestUtils.singletonRecords("test".getBytes) - val logConfig = createLogConfig(segmentMs = 1 * 60 * 60L) + val logConfig = LogTest.createLogConfig(segmentMs = 1 * 60 * 60L) // create a log val log = createLog(logDir, logConfig, maxProducerIdExpirationMs = 24 * 60) @@ -138,6 +143,52 @@ class LogTest { assertEquals("Appending an empty message set should not roll log even if sufficient time has passed.", numSegments, log.numberOfSegments) } + @Test + def testRollSegmentThatAlreadyExists() { + val logConfig = LogTest.createLogConfig(segmentMs = 1 * 60 * 60L) + + // create a log + val log = createLog(logDir, logConfig) + assertEquals("Log begins with a single empty segment.", 1, log.numberOfSegments) + + // roll active segment with the same base offset of size zero should recreate the segment + log.roll(Some(0L)) + assertEquals("Expect 1 segment after roll() empty segment with base offset.", 1, log.numberOfSegments) + + // should be able to append records to active segment + val records = TestUtils.records( + List(new SimpleRecord(mockTime.milliseconds, "k1".getBytes, "v1".getBytes)), + baseOffset = 0L, partitionLeaderEpoch = 0) + log.appendAsFollower(records) + assertEquals("Expect one segment.", 1, log.numberOfSegments) + assertEquals(0L, log.activeSegment.baseOffset) + + // make sure we can append more records + val records2 = TestUtils.records( + List(new SimpleRecord(mockTime.milliseconds + 10, "k2".getBytes, "v2".getBytes)), + baseOffset = 1L, partitionLeaderEpoch = 0) + log.appendAsFollower(records2) + + assertEquals("Expect two records in the log", 2, log.logEndOffset) + assertEquals(0, readLog(log, 0, 100, Some(1)).records.batches.iterator.next().lastOffset) + assertEquals(1, readLog(log, 1, 100, Some(2)).records.batches.iterator.next().lastOffset) + + // roll so that active segment is empty + log.roll() + assertEquals("Expect base offset of active segment to be LEO", 2L, log.activeSegment.baseOffset) + assertEquals("Expect two segments.", 2, log.numberOfSegments) + + // manually resize offset index to force roll of an empty active segment on next append + log.activeSegment.offsetIndex.resize(0) + val records3 = TestUtils.records( + List(new SimpleRecord(mockTime.milliseconds + 12, "k3".getBytes, "v3".getBytes)), + baseOffset = 2L, partitionLeaderEpoch = 0) + log.appendAsFollower(records3) + assertTrue(log.activeSegment.offsetIndex.maxEntries > 1) + assertEquals(2, readLog(log, 2, 100, Some(3)).records.batches.iterator.next().lastOffset) + assertEquals("Expect two segments.", 2, log.numberOfSegments) + } + @Test(expected = classOf[OutOfOrderSequenceException]) def testNonSequentialAppend(): Unit = { // create a log @@ -178,7 +229,7 @@ class LogTest { def testInitializationOfProducerSnapshotsUpgradePath(): Unit = { // simulate the upgrade path by creating a new log with several segments, deleting the // snapshot files, and then reloading the log - val logConfig = createLogConfig(segmentBytes = 64 * 10) + val logConfig = LogTest.createLogConfig(segmentBytes = 64 * 10) var log = createLog(logDir, logConfig) assertEquals(None, log.oldestProducerSnapshotOffset) @@ -204,8 +255,6 @@ class LogTest { // Reload after unclean shutdown with recoveryPoint set to log end offset log = createLog(logDir, logConfig, recoveryPoint = logEndOffset) - // Note that we don't maintain the guarantee of having a snapshot for the 2 most recent segments in this case - expectedSnapshotOffsets = Vector(log.logSegments.last.baseOffset, log.logEndOffset) assertEquals(expectedSnapshotOffsets, listProducerSnapshotOffsets) log.close() @@ -213,15 +262,65 @@ class LogTest { // Reload after unclean shutdown with recoveryPoint set to 0 log = createLog(logDir, logConfig, recoveryPoint = 0L) - // Is this working as intended? + // We progressively create a snapshot for each segment after the recovery point expectedSnapshotOffsets = log.logSegments.map(_.baseOffset).tail.toVector :+ log.logEndOffset assertEquals(expectedSnapshotOffsets, listProducerSnapshotOffsets) log.close() } @Test - def testProducerSnapshotsRecoveryAfterUncleanShutdown(): Unit = { - val logConfig = createLogConfig(segmentBytes = 64 * 10) + def testProducerSnapshotsRecoveryAfterUncleanShutdownV1(): Unit = { + testProducerSnapshotsRecoveryAfterUncleanShutdown(ApiVersion.minSupportedFor(RecordVersion.V1).version) + } + + @Test + def testProducerSnapshotsRecoveryAfterUncleanShutdownCurrentMessageFormat(): Unit = { + testProducerSnapshotsRecoveryAfterUncleanShutdown(ApiVersion.latestVersion.version) + } + + @Test + def testLogReinitializeAfterManualDelete(): Unit = { + val logConfig = LogTest.createLogConfig() + // simulate a case where log data does not exist but the start offset is non-zero + val log = createLog(logDir, logConfig, logStartOffset = 500) + assertEquals(500, log.logStartOffset) + assertEquals(500, log.logEndOffset) + } + + @Test + def testLogEndLessThanStartAfterReopen(): Unit = { + val logConfig = LogTest.createLogConfig() + var log = createLog(logDir, logConfig) + for (i <- 0 until 5) { + val record = new SimpleRecord(mockTime.milliseconds, i.toString.getBytes) + log.appendAsLeader(TestUtils.records(List(record)), leaderEpoch = 0) + log.roll() + } + assertEquals(6, log.logSegments.size) + + // Increment the log start offset + val startOffset = 4 + log.maybeIncrementLogStartOffset(startOffset) + assertTrue(log.logEndOffset > log.logStartOffset) + + // Append garbage to a segment below the current log start offset + val segmentToForceTruncation = log.logSegments.take(2).last + val bw = new BufferedWriter(new FileWriter(segmentToForceTruncation.log.file)) + bw.write("corruptRecord") + bw.close() + log.close() + + // Reopen the log. This will cause truncate the segment to which we appended garbage and delete all other segments. + // All remaining segments will be lower than the current log start offset, which will force deletion of all segments + // and recreation of a single, active segment starting at logStartOffset. + log = createLog(logDir, logConfig, logStartOffset = startOffset) + assertEquals(1, log.logSegments.size) + assertEquals(startOffset, log.logStartOffset) + assertEquals(startOffset, log.logEndOffset) + } + + private def testProducerSnapshotsRecoveryAfterUncleanShutdown(messageFormatVersion: String): Unit = { + val logConfig = LogTest.createLogConfig(segmentBytes = 64 * 10, messageFormatVersion = messageFormatVersion) var log = createLog(logDir, logConfig) assertEquals(None, log.oldestProducerSnapshotOffset) @@ -238,13 +337,23 @@ class LogTest { // 1 segment. We collect the data before closing the log. val offsetForSegmentAfterRecoveryPoint = segmentOffsets(segmentOffsets.size - 3) val offsetForRecoveryPointSegment = segmentOffsets(segmentOffsets.size - 4) - val (segOffsetsBeforeRecovery, segOffsetsAfterRecovery) = segmentOffsets.partition(_ < offsetForRecoveryPointSegment) + val (segOffsetsBeforeRecovery, segOffsetsAfterRecovery) = segmentOffsets.toSet.partition(_ < offsetForRecoveryPointSegment) val recoveryPoint = offsetForRecoveryPointSegment + 1 assertTrue(recoveryPoint < offsetForSegmentAfterRecoveryPoint) log.close() - val segmentsWithReads = ArrayBuffer[LogSegment]() - val recoveredSegments = ArrayBuffer[LogSegment]() + val segmentsWithReads = mutable.Set[LogSegment]() + val recoveredSegments = mutable.Set[LogSegment]() + val expectedSegmentsWithReads = mutable.Set[Long]() + val expectedSnapshotOffsets = mutable.Set[Long]() + + if (logConfig.messageFormatVersion < KAFKA_0_11_0_IV0) { + expectedSegmentsWithReads += activeSegmentOffset + expectedSnapshotOffsets ++= log.logSegments.map(_.baseOffset).toVector.takeRight(2) :+ log.logEndOffset + } else { + expectedSegmentsWithReads ++= segOffsetsBeforeRecovery ++ Set(activeSegmentOffset) + expectedSnapshotOffsets ++= log.logSegments.map(_.baseOffset).toVector.takeRight(4) :+ log.logEndOffset + } def createLogWithInterceptedReads(recoveryPoint: Long) = { val maxProducerIdExpirationMs = 60 * 60 * 1000 @@ -267,7 +376,7 @@ class LogTest { } override def recover(producerStateManager: ProducerStateManager, - leaderEpochCache: Option[LeaderEpochCache]): Int = { + leaderEpochCache: Option[LeaderEpochFileCache]): Int = { recoveredSegments += this super.recover(producerStateManager, leaderEpochCache) } @@ -281,10 +390,9 @@ class LogTest { ProducerStateManager.deleteSnapshotsBefore(logDir, segmentOffsets(segmentOffsets.size - 2)) log = createLogWithInterceptedReads(offsetForRecoveryPointSegment) // We will reload all segments because the recovery point is behind the producer snapshot files (pre KAFKA-5829 behaviour) - assertEquals(segOffsetsBeforeRecovery, segmentsWithReads.map(_.baseOffset) -- Seq(activeSegmentOffset)) + assertEquals(expectedSegmentsWithReads, segmentsWithReads.map(_.baseOffset)) assertEquals(segOffsetsAfterRecovery, recoveredSegments.map(_.baseOffset)) - var expectedSnapshotOffsets = segmentOffsets.takeRight(4) :+ log.logEndOffset - assertEquals(expectedSnapshotOffsets, listProducerSnapshotOffsets) + assertEquals(expectedSnapshotOffsets, listProducerSnapshotOffsets.toSet) log.close() segmentsWithReads.clear() recoveredSegments.clear() @@ -293,15 +401,14 @@ class LogTest { // avoid reading all segments ProducerStateManager.deleteSnapshotsBefore(logDir, offsetForRecoveryPointSegment) log = createLogWithInterceptedReads(recoveryPoint = recoveryPoint) - assertEquals(Seq(activeSegmentOffset), segmentsWithReads.map(_.baseOffset)) + assertEquals(Set(activeSegmentOffset), segmentsWithReads.map(_.baseOffset)) assertEquals(segOffsetsAfterRecovery, recoveredSegments.map(_.baseOffset)) - expectedSnapshotOffsets = log.logSegments.map(_.baseOffset).toVector.takeRight(4) :+ log.logEndOffset - assertEquals(expectedSnapshotOffsets, listProducerSnapshotOffsets) + assertEquals(expectedSnapshotOffsets, listProducerSnapshotOffsets.toSet) // Verify that we keep 2 snapshot files if we checkpoint the log end offset log.deleteSnapshotsAfterRecoveryPointCheckpoint() - expectedSnapshotOffsets = log.logSegments.map(_.baseOffset).toVector.takeRight(2) :+ log.logEndOffset - assertEquals(expectedSnapshotOffsets, listProducerSnapshotOffsets) + val expectedSnapshotsAfterDelete = log.logSegments.map(_.baseOffset).toVector.takeRight(2) :+ log.logEndOffset + assertEquals(expectedSnapshotsAfterDelete, listProducerSnapshotOffsets) log.close() } @@ -320,7 +427,7 @@ class LogTest { @Test def testProducerIdMapOffsetUpdatedForNonIdempotentData() { - val logConfig = createLogConfig(segmentBytes = 2048 * 5) + val logConfig = LogTest.createLogConfig(segmentBytes = 2048 * 5) val log = createLog(logDir, logConfig) val records = TestUtils.records(List(new SimpleRecord(mockTime.milliseconds, "key".getBytes, "value".getBytes))) log.appendAsLeader(records, leaderEpoch = 0) @@ -396,6 +503,9 @@ class LogTest { // We skip directly to updating the map end offset stateManager.updateMapEndOffset(1L) EasyMock.expectLastCall() + // Finally, we take a snapshot + stateManager.takeSnapshot() + EasyMock.expectLastCall().once() EasyMock.replay(stateManager) @@ -408,14 +518,18 @@ class LogTest { def testSkipTruncateAndReloadIfOldMessageFormatAndNoCleanShutdown(): Unit = { val stateManager = EasyMock.mock(classOf[ProducerStateManager]) - EasyMock.expect(stateManager.latestSnapshotOffset).andReturn(None) - stateManager.updateMapEndOffset(0L) EasyMock.expectLastCall().anyTimes() stateManager.takeSnapshot() EasyMock.expectLastCall().anyTimes() + EasyMock.expect(stateManager.isEmpty).andReturn(true) + EasyMock.expectLastCall().once() + + EasyMock.expect(stateManager.firstUnstableOffset).andReturn(None) + EasyMock.expectLastCall().once() + EasyMock.replay(stateManager) val logProps = new Properties() @@ -441,14 +555,18 @@ class LogTest { def testSkipTruncateAndReloadIfOldMessageFormatAndCleanShutdown(): Unit = { val stateManager = EasyMock.mock(classOf[ProducerStateManager]) - EasyMock.expect(stateManager.latestSnapshotOffset).andReturn(None) - stateManager.updateMapEndOffset(0L) EasyMock.expectLastCall().anyTimes() stateManager.takeSnapshot() EasyMock.expectLastCall().anyTimes() + EasyMock.expect(stateManager.isEmpty).andReturn(true) + EasyMock.expectLastCall().once() + + EasyMock.expect(stateManager.firstUnstableOffset).andReturn(None) + EasyMock.expectLastCall().once() + EasyMock.replay(stateManager) val cleanShutdownFile = createCleanShutdownFile() @@ -485,6 +603,12 @@ class LogTest { stateManager.takeSnapshot() EasyMock.expectLastCall().anyTimes() + EasyMock.expect(stateManager.isEmpty).andReturn(true) + EasyMock.expectLastCall().once() + + EasyMock.expect(stateManager.firstUnstableOffset).andReturn(None) + EasyMock.expectLastCall().once() + EasyMock.replay(stateManager) val cleanShutdownFile = createCleanShutdownFile() @@ -511,7 +635,7 @@ class LogTest { @Test def testRebuildProducerIdMapWithCompactedData() { - val logConfig = createLogConfig(segmentBytes = 2048 * 5) + val logConfig = LogTest.createLogConfig(segmentBytes = 2048 * 5) val log = createLog(logDir, logConfig) val pid = 1L val epoch = 0.toShort @@ -554,7 +678,7 @@ class LogTest { @Test def testRebuildProducerStateWithEmptyCompactedBatch() { - val logConfig = createLogConfig(segmentBytes = 2048 * 5) + val logConfig = LogTest.createLogConfig(segmentBytes = 2048 * 5) val log = createLog(logDir, logConfig) val pid = 1L val epoch = 0.toShort @@ -595,7 +719,7 @@ class LogTest { @Test def testUpdateProducerIdMapWithCompactedData() { - val logConfig = createLogConfig(segmentBytes = 2048 * 5) + val logConfig = LogTest.createLogConfig(segmentBytes = 2048 * 5) val log = createLog(logDir, logConfig) val pid = 1L val epoch = 0.toShort @@ -628,7 +752,7 @@ class LogTest { @Test def testProducerIdMapTruncateTo() { - val logConfig = createLogConfig(segmentBytes = 2048 * 5) + val logConfig = LogTest.createLogConfig(segmentBytes = 2048 * 5) val log = createLog(logDir, logConfig) log.appendAsLeader(TestUtils.records(List(new SimpleRecord("a".getBytes))), leaderEpoch = 0) log.appendAsLeader(TestUtils.records(List(new SimpleRecord("b".getBytes))), leaderEpoch = 0) @@ -642,14 +766,18 @@ class LogTest { assertEquals(2, log.latestProducerStateEndOffset) log.truncateTo(1) - assertEquals(None, log.latestProducerSnapshotOffset) + assertEquals(Some(1), log.latestProducerSnapshotOffset) assertEquals(1, log.latestProducerStateEndOffset) + + log.truncateTo(0) + assertEquals(None, log.latestProducerSnapshotOffset) + assertEquals(0, log.latestProducerStateEndOffset) } @Test def testProducerIdMapTruncateToWithNoSnapshots() { // This ensures that the upgrade optimization path cannot be hit after initial loading - val logConfig = createLogConfig(segmentBytes = 2048 * 5) + val logConfig = LogTest.createLogConfig(segmentBytes = 2048 * 5) val log = createLog(logDir, logConfig) val pid = 1L val epoch = 0.toShort @@ -673,7 +801,7 @@ class LogTest { @Test def testLoadProducersAfterDeleteRecordsMidSegment(): Unit = { - val logConfig = createLogConfig(segmentBytes = 2048 * 5) + val logConfig = LogTest.createLogConfig(segmentBytes = 2048 * 5) val log = createLog(logDir, logConfig) val pid1 = 1L val pid2 = 2L @@ -702,7 +830,7 @@ class LogTest { @Test def testLoadProducersAfterDeleteRecordsOnSegment(): Unit = { - val logConfig = createLogConfig(segmentBytes = 2048 * 5) + val logConfig = LogTest.createLogConfig(segmentBytes = 2048 * 5) val log = createLog(logDir, logConfig) val pid1 = 1L val pid2 = 2L @@ -738,7 +866,7 @@ class LogTest { @Test def testProducerIdMapTruncateFullyAndStartAt() { val records = TestUtils.singletonRecords("foo".getBytes) - val logConfig = createLogConfig(segmentBytes = records.sizeInBytes, retentionBytes = records.sizeInBytes * 2) + val logConfig = LogTest.createLogConfig(segmentBytes = records.sizeInBytes, retentionBytes = records.sizeInBytes * 2) val log = createLog(logDir, logConfig) log.appendAsLeader(records, leaderEpoch = 0) log.takeProducerSnapshot() @@ -761,7 +889,7 @@ class LogTest { def testProducerIdExpirationOnSegmentDeletion() { val pid1 = 1L val records = TestUtils.records(Seq(new SimpleRecord("foo".getBytes)), producerId = pid1, producerEpoch = 0, sequence = 0) - val logConfig = createLogConfig(segmentBytes = records.sizeInBytes, retentionBytes = records.sizeInBytes * 2) + val logConfig = LogTest.createLogConfig(segmentBytes = records.sizeInBytes, retentionBytes = records.sizeInBytes * 2) val log = createLog(logDir, logConfig) log.appendAsLeader(records, leaderEpoch = 0) log.takeProducerSnapshot() @@ -785,20 +913,20 @@ class LogTest { @Test def testTakeSnapshotOnRollAndDeleteSnapshotOnRecoveryPointCheckpoint() { - val logConfig = createLogConfig(segmentBytes = 2048 * 5) + val logConfig = LogTest.createLogConfig(segmentBytes = 2048 * 5) val log = createLog(logDir, logConfig) log.appendAsLeader(TestUtils.singletonRecords("a".getBytes), leaderEpoch = 0) - log.roll(1L) + log.roll(Some(1L)) assertEquals(Some(1L), log.latestProducerSnapshotOffset) assertEquals(Some(1L), log.oldestProducerSnapshotOffset) log.appendAsLeader(TestUtils.singletonRecords("b".getBytes), leaderEpoch = 0) - log.roll(2L) + log.roll(Some(2L)) assertEquals(Some(2L), log.latestProducerSnapshotOffset) assertEquals(Some(1L), log.oldestProducerSnapshotOffset) log.appendAsLeader(TestUtils.singletonRecords("c".getBytes), leaderEpoch = 0) - log.roll(3L) + log.roll(Some(3L)) assertEquals(Some(3L), log.latestProducerSnapshotOffset) // roll triggers a flush at the starting offset of the new segment, we should retain all snapshots @@ -818,7 +946,7 @@ class LogTest { @Test def testProducerSnapshotAfterSegmentRollOnAppend(): Unit = { val producerId = 1L - val logConfig = createLogConfig(segmentBytes = 1024) + val logConfig = LogTest.createLogConfig(segmentBytes = 1024) val log = createLog(logDir, logConfig) log.appendAsLeader(TestUtils.records(Seq(new SimpleRecord(mockTime.milliseconds(), new Array[Byte](512))), @@ -850,7 +978,7 @@ class LogTest { @Test def testRebuildTransactionalState(): Unit = { - val logConfig = createLogConfig(segmentBytes = 1024 * 1024 * 5) + val logConfig = LogTest.createLogConfig(segmentBytes = 1024 * 1024 * 5) val log = createLog(logDir, logConfig) val pid = 137L @@ -893,7 +1021,7 @@ class LogTest { val producerIdExpirationCheckIntervalMs = 100 val pid = 23L - val logConfig = createLogConfig(segmentBytes = 2048 * 5) + val logConfig = LogTest.createLogConfig(segmentBytes = 2048 * 5) val log = createLog(logDir, logConfig, maxProducerIdExpirationMs = maxProducerIdExpirationMs, producerIdExpirationCheckIntervalMs = producerIdExpirationCheckIntervalMs) val records = Seq(new SimpleRecord(mockTime.milliseconds(), "foo".getBytes)) @@ -1030,7 +1158,7 @@ class LogTest { @Test def testDuplicateAppendToFollower() : Unit = { - val logConfig = createLogConfig(segmentBytes = 1024 * 1024 * 5) + val logConfig = LogTest.createLogConfig(segmentBytes = 1024 * 1024 * 5) val log = createLog(logDir, logConfig) val epoch: Short = 0 val pid = 1L @@ -1051,7 +1179,7 @@ class LogTest { @Test def testMultipleProducersWithDuplicatesInSingleAppend() : Unit = { - val logConfig = createLogConfig(segmentBytes = 1024 * 1024 * 5) + val logConfig = LogTest.createLogConfig(segmentBytes = 1024 * 1024 * 5) val log = createLog(logDir, logConfig) val pid1 = 1L @@ -1125,7 +1253,7 @@ class LogTest { var set = TestUtils.singletonRecords(value = "test".getBytes, timestamp = mockTime.milliseconds) val maxJitter = 20 * 60L // create a log - val logConfig = createLogConfig(segmentMs = 1 * 60 * 60L, segmentJitterMs = maxJitter) + val logConfig = LogTest.createLogConfig(segmentMs = 1 * 60 * 60L, segmentJitterMs = maxJitter) val log = createLog(logDir, logConfig) assertEquals("Log begins with a single empty segment.", 1, log.numberOfSegments) log.appendAsLeader(set, leaderEpoch = 0) @@ -1150,7 +1278,7 @@ class LogTest { val msgPerSeg = 10 val segmentSize = msgPerSeg * (setSize - 1) // each segment will be 10 messages // create a log - val logConfig = createLogConfig(segmentBytes = segmentSize) + val logConfig = LogTest.createLogConfig(segmentBytes = segmentSize) val log = createLog(logDir, logConfig) assertEquals("There should be exactly 1 segment.", 1, log.numberOfSegments) @@ -1175,7 +1303,7 @@ class LogTest { */ @Test def testAppendAndReadWithSequentialOffsets() { - val logConfig = createLogConfig(segmentBytes = 71) + val logConfig = LogTest.createLogConfig(segmentBytes = 71) val log = createLog(logDir, logConfig) val values = (0 until 100 by 2).map(id => id.toString.getBytes).toArray @@ -1199,7 +1327,7 @@ class LogTest { */ @Test def testAppendAndReadWithNonSequentialOffsets() { - val logConfig = createLogConfig(segmentBytes = 72) + val logConfig = LogTest.createLogConfig(segmentBytes = 72) val log = createLog(logDir, logConfig) val messageIds = ((0 until 50) ++ (50 until 200 by 7)).toArray val records = messageIds.map(id => new SimpleRecord(id.toString.getBytes)) @@ -1223,7 +1351,7 @@ class LogTest { */ @Test def testReadAtLogGap() { - val logConfig = createLogConfig(segmentBytes = 300) + val logConfig = LogTest.createLogConfig(segmentBytes = 300) val log = createLog(logDir, logConfig) // keep appending until we have two segments with only a single message in the second segment @@ -1239,15 +1367,15 @@ class LogTest { @Test(expected = classOf[KafkaStorageException]) def testLogRollAfterLogHandlerClosed() { - val logConfig = createLogConfig() + val logConfig = LogTest.createLogConfig() val log = createLog(logDir, logConfig) log.closeHandlers() - log.roll(1) + log.roll(Some(1L)) } @Test def testReadWithMinMessage() { - val logConfig = createLogConfig(segmentBytes = 72) + val logConfig = LogTest.createLogConfig(segmentBytes = 72) val log = createLog(logDir, logConfig) val messageIds = ((0 until 50) ++ (50 until 200 by 7)).toArray val records = messageIds.map(id => new SimpleRecord(id.toString.getBytes)) @@ -1274,7 +1402,7 @@ class LogTest { @Test def testReadWithTooSmallMaxLength() { - val logConfig = createLogConfig(segmentBytes = 72) + val logConfig = LogTest.createLogConfig(segmentBytes = 72) val log = createLog(logDir, logConfig) val messageIds = ((0 until 50) ++ (50 until 200 by 7)).toArray val records = messageIds.map(id => new SimpleRecord(id.toString.getBytes)) @@ -1308,7 +1436,7 @@ class LogTest { def testReadOutOfRange() { createEmptyLogs(logDir, 1024) // set up replica log starting with offset 1024 and with one message (at offset 1024) - val logConfig = createLogConfig(segmentBytes = 1024) + val logConfig = LogTest.createLogConfig(segmentBytes = 1024) val log = createLog(logDir, logConfig) log.appendAsLeader(TestUtils.singletonRecords(value = "42".getBytes), leaderEpoch = 0) @@ -1340,7 +1468,7 @@ class LogTest { @Test def testLogRolls() { /* create a multipart log with 100 messages */ - val logConfig = createLogConfig(segmentBytes = 100) + val logConfig = LogTest.createLogConfig(segmentBytes = 100) val log = createLog(logDir, logConfig) val numMessages = 100 val messageSets = (0 until numMessages).map(i => TestUtils.singletonRecords(value = i.toString.getBytes, @@ -1378,7 +1506,7 @@ class LogTest { @Test def testCompressedMessages() { /* this log should roll after every messageset */ - val logConfig = createLogConfig(segmentBytes = 110) + val logConfig = LogTest.createLogConfig(segmentBytes = 110) val log = createLog(logDir, logConfig) /* append 2 compressed message sets, each with two messages giving offsets 0, 1, 2, 3 */ @@ -1402,7 +1530,7 @@ class LogTest { for(messagesToAppend <- List(0, 1, 25)) { logDir.mkdirs() // first test a log segment starting at 0 - val logConfig = createLogConfig(segmentBytes = 100, retentionMs = 0) + val logConfig = LogTest.createLogConfig(segmentBytes = 100, retentionMs = 0) val log = createLog(logDir, logConfig) for(i <- 0 until messagesToAppend) log.appendAsLeader(TestUtils.singletonRecords(value = i.toString.getBytes, timestamp = mockTime.milliseconds - 10), leaderEpoch = 0) @@ -1436,7 +1564,7 @@ class LogTest { val messageSet = MemoryRecords.withRecords(CompressionType.NONE, new SimpleRecord("You".getBytes), new SimpleRecord("bethe".getBytes)) // append messages to log val configSegmentSize = messageSet.sizeInBytes - 1 - val logConfig = createLogConfig(segmentBytes = configSegmentSize) + val logConfig = LogTest.createLogConfig(segmentBytes = configSegmentSize) val log = createLog(logDir, logConfig) try { @@ -1461,7 +1589,7 @@ class LogTest { val messageSetWithKeyedMessage = MemoryRecords.withRecords(CompressionType.NONE, keyedMessage) val messageSetWithKeyedMessages = MemoryRecords.withRecords(CompressionType.NONE, keyedMessage, anotherKeyedMessage) - val logConfig = createLogConfig(cleanupPolicy = LogConfig.Compact) + val logConfig = LogTest.createLogConfig(cleanupPolicy = LogConfig.Compact) val log = createLog(logDir, logConfig) try { @@ -1502,7 +1630,7 @@ class LogTest { // append messages to log val maxMessageSize = second.sizeInBytes - 1 - val logConfig = createLogConfig(maxMessageBytes = maxMessageSize) + val logConfig = LogTest.createLogConfig(maxMessageBytes = maxMessageSize) val log = createLog(logDir, logConfig) // should be able to append the small message @@ -1524,7 +1652,7 @@ class LogTest { val messageSize = 100 val segmentSize = 7 * messageSize val indexInterval = 3 * messageSize - val logConfig = createLogConfig(segmentBytes = segmentSize, indexIntervalBytes = indexInterval, segmentIndexBytes = 4096) + val logConfig = LogTest.createLogConfig(segmentBytes = segmentSize, indexIntervalBytes = indexInterval, segmentIndexBytes = 4096) var log = createLog(logDir, logConfig) for(i <- 0 until numMessages) log.appendAsLeader(TestUtils.singletonRecords(value = TestUtils.randomBytes(messageSize), @@ -1568,7 +1696,7 @@ class LogTest { @Test def testBuildTimeIndexWhenNotAssigningOffsets() { val numMessages = 100 - val logConfig = createLogConfig(segmentBytes = 10000, indexIntervalBytes = 1) + val logConfig = LogTest.createLogConfig(segmentBytes = 10000, indexIntervalBytes = 1) val log = createLog(logDir, logConfig) val messages = (0 until numMessages).map { i => @@ -1588,7 +1716,7 @@ class LogTest { def testIndexRebuild() { // publish the messages and close the log val numMessages = 200 - val logConfig = createLogConfig(segmentBytes = 200, indexIntervalBytes = 1) + val logConfig = LogTest.createLogConfig(segmentBytes = 200, indexIntervalBytes = 1) var log = createLog(logDir, logConfig) for(i <- 0 until numMessages) log.appendAsLeader(TestUtils.singletonRecords(value = TestUtils.randomBytes(10), timestamp = mockTime.milliseconds + i * 10), leaderEpoch = 0) @@ -1622,7 +1750,7 @@ class LogTest { def testRebuildTimeIndexForOldMessages() { val numMessages = 200 val segmentSize = 200 - val logConfig = createLogConfig(segmentBytes = segmentSize, indexIntervalBytes = 1, messageFormatVersion = "0.9.0") + val logConfig = LogTest.createLogConfig(segmentBytes = segmentSize, indexIntervalBytes = 1, messageFormatVersion = "0.9.0") var log = createLog(logDir, logConfig) for (i <- 0 until numMessages) log.appendAsLeader(TestUtils.singletonRecords(value = TestUtils.randomBytes(10), @@ -1648,7 +1776,7 @@ class LogTest { def testCorruptIndexRebuild() { // publish the messages and close the log val numMessages = 200 - val logConfig = createLogConfig(segmentBytes = 200, indexIntervalBytes = 1) + val logConfig = LogTest.createLogConfig(segmentBytes = 200, indexIntervalBytes = 1) var log = createLog(logDir, logConfig) for(i <- 0 until numMessages) log.appendAsLeader(TestUtils.singletonRecords(value = TestUtils.randomBytes(10), timestamp = mockTime.milliseconds + i * 10), leaderEpoch = 0) @@ -1694,7 +1822,7 @@ class LogTest { val segmentSize = msgPerSeg * setSize // each segment will be 10 messages // create a log - val logConfig = createLogConfig(segmentBytes = segmentSize) + val logConfig = LogTest.createLogConfig(segmentBytes = segmentSize) val log = createLog(logDir, logConfig) assertEquals("There should be exactly 1 segment.", 1, log.numberOfSegments) @@ -1746,7 +1874,7 @@ class LogTest { val setSize = TestUtils.singletonRecords(value = "test".getBytes, timestamp = mockTime.milliseconds).sizeInBytes val msgPerSeg = 10 val segmentSize = msgPerSeg * setSize // each segment will be 10 messages - val logConfig = createLogConfig(segmentBytes = segmentSize, indexIntervalBytes = setSize - 1) + val logConfig = LogTest.createLogConfig(segmentBytes = segmentSize, indexIntervalBytes = setSize - 1) val log = createLog(logDir, logConfig) assertEquals("There should be exactly 1 segment.", 1, log.numberOfSegments) @@ -1785,7 +1913,7 @@ class LogTest { val bogusTimeIndex2 = Log.timeIndexFile(logDir, 5) def createRecords = TestUtils.singletonRecords(value = "test".getBytes, timestamp = mockTime.milliseconds) - val logConfig = createLogConfig(segmentBytes = createRecords.sizeInBytes * 5, segmentIndexBytes = 1000, indexIntervalBytes = 1) + val logConfig = LogTest.createLogConfig(segmentBytes = createRecords.sizeInBytes * 5, segmentIndexBytes = 1000, indexIntervalBytes = 1) val log = createLog(logDir, logConfig) assertTrue("The first index file should have been replaced with a larger file", bogusIndex1.length > 0) @@ -1807,7 +1935,7 @@ class LogTest { def testReopenThenTruncate() { def createRecords = TestUtils.singletonRecords(value = "test".getBytes, timestamp = mockTime.milliseconds) // create a log - val logConfig = createLogConfig(segmentBytes = createRecords.sizeInBytes * 5, segmentIndexBytes = 1000, indexIntervalBytes = 10000) + val logConfig = LogTest.createLogConfig(segmentBytes = createRecords.sizeInBytes * 5, segmentIndexBytes = 1000, indexIntervalBytes = 10000) var log = createLog(logDir, logConfig) // add enough messages to roll over several segments then close and re-open and attempt to truncate @@ -1827,7 +1955,7 @@ class LogTest { def testAsyncDelete() { def createRecords = TestUtils.singletonRecords(value = "test".getBytes, timestamp = mockTime.milliseconds - 1000L) val asyncDeleteMs = 1000 - val logConfig = createLogConfig(segmentBytes = createRecords.sizeInBytes * 5, segmentIndexBytes = 1000, indexIntervalBytes = 10000, + val logConfig = LogTest.createLogConfig(segmentBytes = createRecords.sizeInBytes * 5, segmentIndexBytes = 1000, indexIntervalBytes = 10000, retentionMs = 999, fileDeleteDelayMs = asyncDeleteMs) val log = createLog(logDir, logConfig) @@ -1861,7 +1989,7 @@ class LogTest { @Test def testOpenDeletesObsoleteFiles() { def createRecords = TestUtils.singletonRecords(value = "test".getBytes, timestamp = mockTime.milliseconds - 1000) - val logConfig = createLogConfig(segmentBytes = createRecords.sizeInBytes * 5, segmentIndexBytes = 1000, retentionMs = 999) + val logConfig = LogTest.createLogConfig(segmentBytes = createRecords.sizeInBytes * 5, segmentIndexBytes = 1000, retentionMs = 999) var log = createLog(logDir, logConfig) // append some messages to create some segments @@ -1885,13 +2013,72 @@ class LogTest { assertTrue("Message payload should be null.", !head.hasValue) } - @Test(expected = classOf[IllegalArgumentException]) + @Test def testAppendWithOutOfOrderOffsetsThrowsException() { val log = createLog(logDir, LogConfig()) + + val appendOffsets = Seq(0L, 1L, 3L, 2L, 4L) + val buffer = ByteBuffer.allocate(512) + for (offset <- appendOffsets) { + val builder = MemoryRecords.builder(buffer, RecordBatch.MAGIC_VALUE_V2, CompressionType.NONE, + TimestampType.LOG_APPEND_TIME, offset, mockTime.milliseconds(), + 1L, 0, 0, false, 0) + builder.append(new SimpleRecord("key".getBytes, "value".getBytes)) + builder.close() + } + buffer.flip() + val memoryRecords = MemoryRecords.readableRecords(buffer) + + assertThrows[OffsetsOutOfOrderException] { + log.appendAsFollower(memoryRecords) + } + } + + @Test + def testAppendBelowExpectedOffsetThrowsException() { + val log = createLog(logDir, LogConfig()) val records = (0 until 2).map(id => new SimpleRecord(id.toString.getBytes)).toArray records.foreach(record => log.appendAsLeader(MemoryRecords.withRecords(CompressionType.NONE, record), leaderEpoch = 0)) - val invalidRecord = MemoryRecords.withRecords(CompressionType.NONE, new SimpleRecord(1.toString.getBytes)) - log.appendAsFollower(invalidRecord) + + val magicVals = Seq(RecordBatch.MAGIC_VALUE_V0, RecordBatch.MAGIC_VALUE_V1, RecordBatch.MAGIC_VALUE_V2) + val compressionTypes = Seq(CompressionType.NONE, CompressionType.LZ4) + for (magic <- magicVals; compression <- compressionTypes) { + val invalidRecord = MemoryRecords.withRecords(magic, compression, new SimpleRecord(1.toString.getBytes)) + withClue(s"Magic=$magic, compressionType=$compression") { + assertThrows[UnexpectedAppendOffsetException] { + log.appendAsFollower(invalidRecord) + } + } + } + } + + @Test + def testAppendEmptyLogBelowLogStartOffsetThrowsException() { + createEmptyLogs(logDir, 7) + val log = createLog(logDir, LogConfig(), brokerTopicStats = brokerTopicStats) + assertEquals(7L, log.logStartOffset) + assertEquals(7L, log.logEndOffset) + + val firstOffset = 4L + val magicVals = Seq(RecordBatch.MAGIC_VALUE_V0, RecordBatch.MAGIC_VALUE_V1, RecordBatch.MAGIC_VALUE_V2) + val compressionTypes = Seq(CompressionType.NONE, CompressionType.LZ4) + for (magic <- magicVals; compression <- compressionTypes) { + val batch = TestUtils.records(List(new SimpleRecord("k1".getBytes, "v1".getBytes), + new SimpleRecord("k2".getBytes, "v2".getBytes), + new SimpleRecord("k3".getBytes, "v3".getBytes)), + magicValue = magic, codec = compression, + baseOffset = firstOffset) + + withClue(s"Magic=$magic, compressionType=$compression") { + val exception = intercept[UnexpectedAppendOffsetException] { + log.appendAsFollower(records = batch) + } + assertEquals(s"Magic=$magic, compressionType=$compression, UnexpectedAppendOffsetException#firstOffset", + firstOffset, exception.firstOffset) + assertEquals(s"Magic=$magic, compressionType=$compression, UnexpectedAppendOffsetException#lastOffset", + firstOffset + 2, exception.lastOffset) + } + } } @Test @@ -1904,7 +2091,7 @@ class LogTest { @Test def testCorruptLog() { // append some messages to create some segments - val logConfig = createLogConfig(segmentBytes = 1000, indexIntervalBytes = 1, maxMessageBytes = 64 * 1024) + val logConfig = LogTest.createLogConfig(segmentBytes = 1000, indexIntervalBytes = 1, maxMessageBytes = 64 * 1024) def createRecords = TestUtils.singletonRecords(value = "test".getBytes, timestamp = mockTime.milliseconds) val recoveryPoint = 50L for (_ <- 0 until 10) { @@ -1922,7 +2109,7 @@ class LogTest { TestUtils.appendNonsenseToFile(log.activeSegment.log.file, TestUtils.random.nextInt(1024) + 1) // attempt recovery - log = createLog(logDir, logConfig, 0L, recoveryPoint) + log = createLog(logDir, logConfig, brokerTopicStats, 0L, recoveryPoint) assertEquals(numMessages, log.logEndOffset) val recovered = log.logSegments.flatMap(_.log.records.asScala.toList).toList @@ -1943,7 +2130,7 @@ class LogTest { @Test def testOverCompactedLogRecovery(): Unit = { // append some messages to create some segments - val logConfig = createLogConfig(segmentBytes = 1000, indexIntervalBytes = 1, maxMessageBytes = 64 * 1024) + val logConfig = LogTest.createLogConfig(segmentBytes = 1000, indexIntervalBytes = 1, maxMessageBytes = 64 * 1024) val log = createLog(logDir, logConfig) val set1 = MemoryRecords.withRecords(0, CompressionType.NONE, 0, new SimpleRecord("v1".getBytes(), "k1".getBytes())) val set2 = MemoryRecords.withRecords(Integer.MAX_VALUE.toLong + 2, CompressionType.NONE, 0, new SimpleRecord("v3".getBytes(), "k3".getBytes())) @@ -1976,7 +2163,7 @@ class LogTest { @Test def testOverCompactedLogRecoveryMultiRecord(): Unit = { // append some messages to create some segments - val logConfig = createLogConfig(segmentBytes = 1000, indexIntervalBytes = 1, maxMessageBytes = 64 * 1024) + val logConfig = LogTest.createLogConfig(segmentBytes = 1000, indexIntervalBytes = 1, maxMessageBytes = 64 * 1024) val log = createLog(logDir, logConfig) val set1 = MemoryRecords.withRecords(0, CompressionType.NONE, 0, new SimpleRecord("v1".getBytes(), "k1".getBytes())) val set2 = MemoryRecords.withRecords(Integer.MAX_VALUE.toLong + 2, CompressionType.GZIP, 0, @@ -2015,7 +2202,7 @@ class LogTest { @Test def testOverCompactedLogRecoveryMultiRecordV1(): Unit = { // append some messages to create some segments - val logConfig = createLogConfig(segmentBytes = 1000, indexIntervalBytes = 1, maxMessageBytes = 64 * 1024) + val logConfig = LogTest.createLogConfig(segmentBytes = 1000, indexIntervalBytes = 1, maxMessageBytes = 64 * 1024) val log = createLog(logDir, logConfig) val set1 = MemoryRecords.withRecords(RecordBatch.MAGIC_VALUE_V1, 0, CompressionType.NONE, new SimpleRecord("v1".getBytes(), "k1".getBytes())) @@ -2053,10 +2240,236 @@ class LogTest { Utils.delete(logDir) } + @Test + def testSplitOnOffsetOverflow(): Unit = { + // create a log such that one log segment has offsets that overflow, and call the split API on that segment + val logConfig = LogTest.createLogConfig(indexIntervalBytes = 1, fileDeleteDelayMs = 1000) + val (log, segmentWithOverflow) = createLogWithOffsetOverflow(logConfig) + assertTrue("At least one segment must have offset overflow", LogTest.hasOffsetOverflow(log)) + + val allRecordsBeforeSplit = LogTest.allRecords(log) + + // split the segment with overflow + log.splitOverflowedSegment(segmentWithOverflow) + + // assert we were successfully able to split the segment + assertEquals(4, log.numberOfSegments) + LogTest.verifyRecordsInLog(log, allRecordsBeforeSplit) + + // verify we do not have offset overflow anymore + assertFalse(LogTest.hasOffsetOverflow(log)) + } + + @Test + def testDegenerateSegmentSplit(): Unit = { + // This tests a scenario where all of the batches appended to a segment have overflowed. + // When we split the overflowed segment, only one new segment will be created. + + val overflowOffset = Int.MaxValue + 1L + val batch1 = MemoryRecords.withRecords(overflowOffset, CompressionType.NONE, 0, + new SimpleRecord("a".getBytes)) + val batch2 = MemoryRecords.withRecords(overflowOffset + 1, CompressionType.NONE, 0, + new SimpleRecord("b".getBytes)) + + testDegenerateSplitSegmentWithOverflow(segmentBaseOffset = 0L, List(batch1, batch2)) + } + + @Test + def testDegenerateSegmentSplitWithOutOfRangeBatchLastOffset(): Unit = { + // Degenerate case where the only batch in the segment overflows. In this scenario, + // the first offset of the batch is valid, but the last overflows. + + val firstBatchBaseOffset = Int.MaxValue - 1 + val records = MemoryRecords.withRecords(firstBatchBaseOffset, CompressionType.NONE, 0, + new SimpleRecord("a".getBytes), + new SimpleRecord("b".getBytes), + new SimpleRecord("c".getBytes)) + + testDegenerateSplitSegmentWithOverflow(segmentBaseOffset = 0L, List(records)) + } + + private def testDegenerateSplitSegmentWithOverflow(segmentBaseOffset: Long, records: List[MemoryRecords]): Unit = { + val segment = LogTest.rawSegment(logDir, segmentBaseOffset) + records.foreach(segment.append _) + segment.close() + + // Create clean shutdown file so that we do not split during the load + createCleanShutdownFile() + + val logConfig = LogTest.createLogConfig(indexIntervalBytes = 1, fileDeleteDelayMs = 1000) + val log = createLog(logDir, logConfig, recoveryPoint = Long.MaxValue) + + val segmentWithOverflow = LogTest.firstOverflowSegment(log).getOrElse { + Assertions.fail("Failed to create log with a segment which has overflowed offsets") + } + + val allRecordsBeforeSplit = LogTest.allRecords(log) + log.splitOverflowedSegment(segmentWithOverflow) + + assertEquals(1, log.numberOfSegments) + + val firstBatchBaseOffset = records.head.batches.asScala.head.baseOffset + assertEquals(firstBatchBaseOffset, log.activeSegment.baseOffset) + LogTest.verifyRecordsInLog(log, allRecordsBeforeSplit) + + assertFalse(LogTest.hasOffsetOverflow(log)) + } + + @Test + def testRecoveryOfSegmentWithOffsetOverflow(): Unit = { + val logConfig = LogTest.createLogConfig(indexIntervalBytes = 1, fileDeleteDelayMs = 1000) + val (log, _) = createLogWithOffsetOverflow(logConfig) + val expectedKeys = LogTest.keysInLog(log) + + // Run recovery on the log. This should split the segment underneath. Ignore .deleted files as we could have still + // have them lying around after the split. + val recoveredLog = recoverAndCheck(logConfig, expectedKeys) + assertEquals(expectedKeys, LogTest.keysInLog(recoveredLog)) + + // Running split again would throw an error + for (segment <- recoveredLog.logSegments) { + try { + log.splitOverflowedSegment(segment) + fail() + } catch { + case _: IllegalArgumentException => + } + } + } + + @Test + def testRecoveryAfterCrashDuringSplitPhase1(): Unit = { + val logConfig = LogTest.createLogConfig(indexIntervalBytes = 1, fileDeleteDelayMs = 1000) + val (log, segmentWithOverflow) = createLogWithOffsetOverflow(logConfig) + val expectedKeys = LogTest.keysInLog(log) + val numSegmentsInitial = log.logSegments.size + + // Split the segment + val newSegments = log.splitOverflowedSegment(segmentWithOverflow) + + // Simulate recovery just after .cleaned file is created, before rename to .swap. On recovery, existing split + // operation is aborted but the recovery process itself kicks off split which should complete. + newSegments.reverse.foreach(segment => { + segment.changeFileSuffixes("", Log.CleanedFileSuffix) + segment.truncateTo(0) + }) + for (file <- logDir.listFiles if file.getName.endsWith(Log.DeletedFileSuffix)) + Utils.atomicMoveWithFallback(file.toPath, Paths.get(CoreUtils.replaceSuffix(file.getPath, Log.DeletedFileSuffix, ""))) + + val recoveredLog = recoverAndCheck(logConfig, expectedKeys) + assertEquals(expectedKeys, LogTest.keysInLog(recoveredLog)) + assertEquals(numSegmentsInitial + 1, recoveredLog.logSegments.size) + recoveredLog.close() + } + + @Test + def testRecoveryAfterCrashDuringSplitPhase2(): Unit = { + val logConfig = LogTest.createLogConfig(indexIntervalBytes = 1, fileDeleteDelayMs = 1000) + val (log, segmentWithOverflow) = createLogWithOffsetOverflow(logConfig) + val expectedKeys = LogTest.keysInLog(log) + val numSegmentsInitial = log.logSegments.size + + // Split the segment + val newSegments = log.splitOverflowedSegment(segmentWithOverflow) + + // Simulate recovery just after one of the new segments has been renamed to .swap. On recovery, existing split + // operation is aborted but the recovery process itself kicks off split which should complete. + newSegments.reverse.foreach { segment => + if (segment != newSegments.last) + segment.changeFileSuffixes("", Log.CleanedFileSuffix) + else + segment.changeFileSuffixes("", Log.SwapFileSuffix) + segment.truncateTo(0) + } + for (file <- logDir.listFiles if file.getName.endsWith(Log.DeletedFileSuffix)) + Utils.atomicMoveWithFallback(file.toPath, Paths.get(CoreUtils.replaceSuffix(file.getPath, Log.DeletedFileSuffix, ""))) + + val recoveredLog = recoverAndCheck(logConfig, expectedKeys) + assertEquals(expectedKeys, LogTest.keysInLog(recoveredLog)) + assertEquals(numSegmentsInitial + 1, recoveredLog.logSegments.size) + recoveredLog.close() + } + + @Test + def testRecoveryAfterCrashDuringSplitPhase3(): Unit = { + val logConfig = LogTest.createLogConfig(indexIntervalBytes = 1, fileDeleteDelayMs = 1000) + val (log, segmentWithOverflow) = createLogWithOffsetOverflow(logConfig) + val expectedKeys = LogTest.keysInLog(log) + val numSegmentsInitial = log.logSegments.size + + // Split the segment + val newSegments = log.splitOverflowedSegment(segmentWithOverflow) + + // Simulate recovery right after all new segments have been renamed to .swap. On recovery, existing split operation + // is completed and the old segment must be deleted. + newSegments.reverse.foreach(segment => { + segment.changeFileSuffixes("", Log.SwapFileSuffix) + }) + for (file <- logDir.listFiles if file.getName.endsWith(Log.DeletedFileSuffix)) + Utils.atomicMoveWithFallback(file.toPath, Paths.get(CoreUtils.replaceSuffix(file.getPath, Log.DeletedFileSuffix, ""))) + + // Truncate the old segment + segmentWithOverflow.truncateTo(0) + + val recoveredLog = recoverAndCheck(logConfig, expectedKeys) + assertEquals(expectedKeys, LogTest.keysInLog(recoveredLog)) + assertEquals(numSegmentsInitial + 1, recoveredLog.logSegments.size) + log.close() + } + + @Test + def testRecoveryAfterCrashDuringSplitPhase4(): Unit = { + val logConfig = LogTest.createLogConfig(indexIntervalBytes = 1, fileDeleteDelayMs = 1000) + val (log, segmentWithOverflow) = createLogWithOffsetOverflow(logConfig) + val expectedKeys = LogTest.keysInLog(log) + val numSegmentsInitial = log.logSegments.size + + // Split the segment + val newSegments = log.splitOverflowedSegment(segmentWithOverflow) + + // Simulate recovery right after all new segments have been renamed to .swap and old segment has been deleted. On + // recovery, existing split operation is completed. + newSegments.reverse.foreach(_.changeFileSuffixes("", Log.SwapFileSuffix)) + + for (file <- logDir.listFiles if file.getName.endsWith(Log.DeletedFileSuffix)) + Utils.delete(file) + + // Truncate the old segment + segmentWithOverflow.truncateTo(0) + + val recoveredLog = recoverAndCheck(logConfig, expectedKeys) + assertEquals(expectedKeys, LogTest.keysInLog(recoveredLog)) + assertEquals(numSegmentsInitial + 1, recoveredLog.logSegments.size) + recoveredLog.close() + } + + @Test + def testRecoveryAfterCrashDuringSplitPhase5(): Unit = { + val logConfig = LogTest.createLogConfig(indexIntervalBytes = 1, fileDeleteDelayMs = 1000) + val (log, segmentWithOverflow) = createLogWithOffsetOverflow(logConfig) + val expectedKeys = LogTest.keysInLog(log) + val numSegmentsInitial = log.logSegments.size + + // Split the segment + val newSegments = log.splitOverflowedSegment(segmentWithOverflow) + + // Simulate recovery right after one of the new segment has been renamed to .swap and the other to .log. On + // recovery, existing split operation is completed. + newSegments.last.changeFileSuffixes("", Log.SwapFileSuffix) + + // Truncate the old segment + segmentWithOverflow.truncateTo(0) + + val recoveredLog = recoverAndCheck(logConfig, expectedKeys) + assertEquals(expectedKeys, LogTest.keysInLog(recoveredLog)) + assertEquals(numSegmentsInitial + 1, recoveredLog.logSegments.size) + recoveredLog.close() + } + @Test def testCleanShutdownFile() { // append some messages to create some segments - val logConfig = createLogConfig(segmentBytes = 1000, indexIntervalBytes = 1, maxMessageBytes = 64 * 1024) + val logConfig = LogTest.createLogConfig(segmentBytes = 1000, indexIntervalBytes = 1, maxMessageBytes = 64 * 1024) def createRecords = TestUtils.singletonRecords(value = "test".getBytes, timestamp = mockTime.milliseconds) val cleanShutdownFile = createCleanShutdownFile() @@ -2228,10 +2641,60 @@ class LogTest { def topicPartitionName(topic: String, partition: String): String = topic + "-" + partition + /** + * Due to KAFKA-7968, we want to make sure that we do not + * make use of old leader epoch cache files when the message format does not support it + */ + @Test + def testOldMessageFormatDeletesEpochCacheIfUnsupported(): Unit = { + def createRecords = TestUtils.singletonRecords(value = "test".getBytes, timestamp = mockTime.milliseconds - 1000) + val epochCacheSupportingConfig = LogTest.createLogConfig(segmentBytes = createRecords.sizeInBytes * 5, segmentIndexBytes = 1000, retentionMs = 999) + + // append some records to create segments and assign some epochs to create epoch files + val log = createLog(logDir, epochCacheSupportingConfig) + for (_ <- 0 until 100) + log.appendAsLeader(createRecords, leaderEpoch = 0) + log.leaderEpochCache.assign(0, 40) + log.leaderEpochCache.assign(1, 90) + assertEquals((1, 100), log.leaderEpochCache.endOffsetFor(1)) + + // instantiate the log with an old format that does not support the leader epoch + val epochCacheNonSupportingConfig = LogTest.createLogConfig(segmentBytes = createRecords.sizeInBytes * 5, segmentIndexBytes = 1000, + retentionMs = 999, messageFormatVersion = "0.10.2") + val log2 = createLog(logDir, epochCacheNonSupportingConfig) + assertLeaderEpochCacheEmpty(log2) + } + + @Test + def testLeaderEpochCacheClearedAfterDynamicMessageFormatDowngrade(): Unit = { + val logConfig = LogTest.createLogConfig(segmentBytes = 1000, indexIntervalBytes = 1, maxMessageBytes = 64 * 1024) + val log = createLog(logDir, logConfig) + log.appendAsLeader(TestUtils.records(List(new SimpleRecord("foo".getBytes()))), leaderEpoch = 5) + assertEquals((5, 1), log.leaderEpochCache.endOffsetFor(5)) + + val downgradedLogConfig = LogTest.createLogConfig(segmentBytes = 1000, indexIntervalBytes = 1, + maxMessageBytes = 64 * 1024, messageFormatVersion = kafka.api.KAFKA_0_10_2_IV0.shortVersion) + log.updateConfig(Set(LogConfig.MessageFormatVersionProp), downgradedLogConfig) + assertLeaderEpochCacheEmpty(log) + + log.appendAsLeader(TestUtils.records(List(new SimpleRecord("bar".getBytes())), + magicValue = RecordVersion.V1.value), leaderEpoch = 5) + assertLeaderEpochCacheEmpty(log) + } + + private def assertLeaderEpochCacheEmpty(log: Log): Unit = { + assertFalse(log.leaderEpochCache.nonEmpty) + + // check that the file is empty as well + val checkpointFile = new LeaderEpochCheckpointFile(LeaderEpochFile.newFile(log.dir)) + val cache = new LeaderEpochFileCache(log.topicPartition, log.logEndOffset _, checkpointFile) + assertFalse(cache.nonEmpty) + } + @Test def testDeleteOldSegments() { def createRecords = TestUtils.singletonRecords(value = "test".getBytes, timestamp = mockTime.milliseconds - 1000) - val logConfig = createLogConfig(segmentBytes = createRecords.sizeInBytes * 5, segmentIndexBytes = 1000, retentionMs = 999) + val logConfig = LogTest.createLogConfig(segmentBytes = createRecords.sizeInBytes * 5, segmentIndexBytes = 1000, retentionMs = 999) val log = createLog(logDir, logConfig) // append some messages to create some segments @@ -2265,8 +2728,8 @@ class LogTest { log.onHighWatermarkIncremented(log.logEndOffset) log.deleteOldSegments() assertEquals("The deleted segments should be gone.", 1, log.numberOfSegments) - assertEquals("Epoch entries should have gone.", 1, epochCache(log).epochEntries().size) - assertEquals("Epoch entry should be the latest epoch and the leo.", EpochEntry(1, 100), epochCache(log).epochEntries().head) + assertEquals("Epoch entries should have gone.", 1, epochCache(log).epochEntries.size) + assertEquals("Epoch entry should be the latest epoch and the leo.", EpochEntry(1, 100), epochCache(log).epochEntries.head) // append some messages to create some segments for (_ <- 0 until 100) @@ -2275,31 +2738,31 @@ class LogTest { log.delete() assertEquals("The number of segments should be 0", 0, log.numberOfSegments) assertEquals("The number of deleted segments should be zero.", 0, log.deleteOldSegments()) - assertEquals("Epoch entries should have gone.", 0, epochCache(log).epochEntries().size) + assertEquals("Epoch entries should have gone.", 0, epochCache(log).epochEntries.size) } @Test def testLogDeletionAfterClose() { def createRecords = TestUtils.singletonRecords(value = "test".getBytes, timestamp = mockTime.milliseconds - 1000) - val logConfig = createLogConfig(segmentBytes = createRecords.sizeInBytes * 5, segmentIndexBytes = 1000, retentionMs = 999) + val logConfig = LogTest.createLogConfig(segmentBytes = createRecords.sizeInBytes * 5, segmentIndexBytes = 1000, retentionMs = 999) val log = createLog(logDir, logConfig) // append some messages to create some segments log.appendAsLeader(createRecords, leaderEpoch = 0) assertEquals("The deleted segments should be gone.", 1, log.numberOfSegments) - assertEquals("Epoch entries should have gone.", 1, epochCache(log).epochEntries().size) + assertEquals("Epoch entries should have gone.", 1, epochCache(log).epochEntries.size) log.close() log.delete() assertEquals("The number of segments should be 0", 0, log.numberOfSegments) - assertEquals("Epoch entries should have gone.", 0, epochCache(log).epochEntries().size) + assertEquals("Epoch entries should have gone.", 0, epochCache(log).epochEntries.size) } @Test def testLogDeletionAfterDeleteRecords() { def createRecords = TestUtils.singletonRecords("test".getBytes) - val logConfig = createLogConfig(segmentBytes = createRecords.sizeInBytes * 5) + val logConfig = LogTest.createLogConfig(segmentBytes = createRecords.sizeInBytes * 5) val log = createLog(logDir, logConfig) for (_ <- 0 until 15) @@ -2331,7 +2794,7 @@ class LogTest { @Test def shouldDeleteSizeBasedSegments() { def createRecords = TestUtils.singletonRecords("test".getBytes) - val logConfig = createLogConfig(segmentBytes = createRecords.sizeInBytes * 5, retentionBytes = createRecords.sizeInBytes * 10) + val logConfig = LogTest.createLogConfig(segmentBytes = createRecords.sizeInBytes * 5, retentionBytes = createRecords.sizeInBytes * 10) val log = createLog(logDir, logConfig) // append some messages to create some segments @@ -2346,7 +2809,7 @@ class LogTest { @Test def shouldNotDeleteSizeBasedSegmentsWhenUnderRetentionSize() { def createRecords = TestUtils.singletonRecords("test".getBytes) - val logConfig = createLogConfig(segmentBytes = createRecords.sizeInBytes * 5, retentionBytes = createRecords.sizeInBytes * 15) + val logConfig = LogTest.createLogConfig(segmentBytes = createRecords.sizeInBytes * 5, retentionBytes = createRecords.sizeInBytes * 15) val log = createLog(logDir, logConfig) // append some messages to create some segments @@ -2361,7 +2824,7 @@ class LogTest { @Test def shouldDeleteTimeBasedSegmentsReadyToBeDeleted() { def createRecords = TestUtils.singletonRecords("test".getBytes, timestamp = 10) - val logConfig = createLogConfig(segmentBytes = createRecords.sizeInBytes * 5, retentionMs = 10000) + val logConfig = LogTest.createLogConfig(segmentBytes = createRecords.sizeInBytes * 5, retentionMs = 10000) val log = createLog(logDir, logConfig) // append some messages to create some segments @@ -2376,7 +2839,7 @@ class LogTest { @Test def shouldNotDeleteTimeBasedSegmentsWhenNoneReadyToBeDeleted() { def createRecords = TestUtils.singletonRecords("test".getBytes, timestamp = mockTime.milliseconds) - val logConfig = createLogConfig(segmentBytes = createRecords.sizeInBytes * 5, retentionMs = 10000000) + val logConfig = LogTest.createLogConfig(segmentBytes = createRecords.sizeInBytes * 5, retentionMs = 10000000) val log = createLog(logDir, logConfig) // append some messages to create some segments @@ -2391,7 +2854,7 @@ class LogTest { @Test def shouldNotDeleteSegmentsWhenPolicyDoesNotIncludeDelete() { def createRecords = TestUtils.singletonRecords("test".getBytes, key = "test".getBytes(), timestamp = 10L) - val logConfig = createLogConfig(segmentBytes = createRecords.sizeInBytes * 5, retentionMs = 10000, cleanupPolicy = "compact") + val logConfig = LogTest.createLogConfig(segmentBytes = createRecords.sizeInBytes * 5, retentionMs = 10000, cleanupPolicy = "compact") val log = createLog(logDir, logConfig) // append some messages to create some segments @@ -2410,7 +2873,7 @@ class LogTest { @Test def shouldDeleteSegmentsReadyToBeDeletedWhenCleanupPolicyIsCompactAndDelete() { def createRecords = TestUtils.singletonRecords("test".getBytes, key = "test".getBytes, timestamp = 10L) - val logConfig = createLogConfig(segmentBytes = createRecords.sizeInBytes * 5, retentionMs = 10000, cleanupPolicy = "compact,delete") + val logConfig = LogTest.createLogConfig(segmentBytes = createRecords.sizeInBytes * 5, retentionMs = 10000, cleanupPolicy = "compact,delete") val log = createLog(logDir, logConfig) // append some messages to create some segments @@ -2466,13 +2929,13 @@ class LogTest { for (i <- records.indices) log.appendAsFollower(recordsForEpoch(i)) - assertEquals(42, log.leaderEpochCache.asInstanceOf[LeaderEpochFileCache].latestEpoch()) + assertEquals(42, log.leaderEpochCache.latestEpoch) } @Test def shouldTruncateLeaderEpochsWhenDeletingSegments() { def createRecords = TestUtils.singletonRecords("test".getBytes) - val logConfig = createLogConfig(segmentBytes = createRecords.sizeInBytes * 5, retentionBytes = createRecords.sizeInBytes * 10) + val logConfig = LogTest.createLogConfig(segmentBytes = createRecords.sizeInBytes * 5, retentionBytes = createRecords.sizeInBytes * 10) val log = createLog(logDir, logConfig) val cache = epochCache(log) @@ -2497,7 +2960,7 @@ class LogTest { @Test def shouldUpdateOffsetForLeaderEpochsWhenDeletingSegments() { def createRecords = TestUtils.singletonRecords("test".getBytes) - val logConfig = createLogConfig(segmentBytes = createRecords.sizeInBytes * 5, retentionBytes = createRecords.sizeInBytes * 10) + val logConfig = LogTest.createLogConfig(segmentBytes = createRecords.sizeInBytes * 5, retentionBytes = createRecords.sizeInBytes * 10) val log = createLog(logDir, logConfig) val cache = epochCache(log) @@ -2521,19 +2984,24 @@ class LogTest { @Test def shouldTruncateLeaderEpochFileWhenTruncatingLog() { - def createRecords = TestUtils.singletonRecords(value = "test".getBytes, timestamp = mockTime.milliseconds) - val logConfig = createLogConfig(segmentBytes = 10 * createRecords.sizeInBytes) + def createRecords(startOffset: Long, epoch: Int): MemoryRecords = { + TestUtils.records(Seq(new SimpleRecord("value".getBytes)), + baseOffset = startOffset, partitionLeaderEpoch = epoch) + } + + val logConfig = LogTest.createLogConfig(segmentBytes = 10 * createRecords(0, 0).sizeInBytes) val log = createLog(logDir, logConfig) val cache = epochCache(log) - //Given 2 segments, 10 messages per segment - for (epoch <- 1 to 20) - log.appendAsLeader(createRecords, leaderEpoch = 0) + def append(epoch: Int, startOffset: Long, count: Int): Unit = { + for (i <- 0 until count) + log.appendAsFollower(createRecords(startOffset + i, epoch)) + } - //Simulate some leader changes at specific offsets - cache.assign(0, 0) - cache.assign(1, 10) - cache.assign(2, 16) + //Given 2 segments, 10 messages per segment + append(epoch = 0, startOffset = 0, count = 10) + append(epoch = 1, startOffset = 10, count = 6) + append(epoch = 2, startOffset = 16, count = 4) assertEquals(2, log.numberOfSegments) assertEquals(20, log.logEndOffset) @@ -2564,8 +3032,8 @@ class LogTest { } /** - * Append a bunch of messages to a log and then re-open it with recovery and check that the leader epochs are recovered properly. - */ + * Append a bunch of messages to a log and then re-open it with recovery and check that the leader epochs are recovered properly. + */ @Test def testLogRecoversForLeaderEpoch() { val log = createLog(logDir, LogConfig()) @@ -2585,7 +3053,7 @@ class LogTest { assertEquals(ListBuffer(EpochEntry(1, 0), EpochEntry(2, 1), EpochEntry(3, 3)), leaderEpochCache.epochEntries) // deliberately remove some of the epoch entries - leaderEpochCache.clearAndFlushLatest(2) + leaderEpochCache.truncateFromEnd(2) assertNotEquals(ListBuffer(EpochEntry(1, 0), EpochEntry(2, 1), EpochEntry(3, 3)), leaderEpochCache.epochEntries) log.close() @@ -2599,15 +3067,15 @@ class LogTest { } /** - * Wrap a single record log buffer with leader epoch. - */ + * Wrap a single record log buffer with leader epoch. + */ private def singletonRecordsWithLeaderEpoch(value: Array[Byte], - key: Array[Byte] = null, - leaderEpoch: Int, - offset: Long, - codec: CompressionType = CompressionType.NONE, - timestamp: Long = RecordBatch.NO_TIMESTAMP, - magicValue: Byte = RecordBatch.CURRENT_MAGIC_VALUE): MemoryRecords = { + key: Array[Byte] = null, + leaderEpoch: Int, + offset: Long, + codec: CompressionType = CompressionType.NONE, + timestamp: Long = RecordBatch.NO_TIMESTAMP, + magicValue: Byte = RecordBatch.CURRENT_MAGIC_VALUE): MemoryRecords = { val records = Seq(new SimpleRecord(timestamp, key, value)) val buf = ByteBuffer.allocate(DefaultRecordBatch.sizeInBytes(records.asJava)) @@ -2618,7 +3086,7 @@ class LogTest { } def testFirstUnstableOffsetNoTransactionalData() { - val logConfig = createLogConfig(segmentBytes = 1024 * 1024 * 5) + val logConfig = LogTest.createLogConfig(segmentBytes = 1024 * 1024 * 5) val log = createLog(logDir, logConfig) val records = MemoryRecords.withRecords(CompressionType.NONE, @@ -2632,7 +3100,7 @@ class LogTest { @Test def testFirstUnstableOffsetWithTransactionalData() { - val logConfig = createLogConfig(segmentBytes = 1024 * 1024 * 5) + val logConfig = LogTest.createLogConfig(segmentBytes = 1024 * 1024 * 5) val log = createLog(logDir, logConfig) val pid = 137L @@ -2670,7 +3138,7 @@ class LogTest { @Test def testTransactionIndexUpdated(): Unit = { - val logConfig = createLogConfig(segmentBytes = 1024 * 1024 * 5) + val logConfig = LogTest.createLogConfig(segmentBytes = 1024 * 1024 * 5) val log = createLog(logDir, logConfig) val epoch = 0.toShort @@ -2711,7 +3179,7 @@ class LogTest { @Test def testFullTransactionIndexRecovery(): Unit = { - val logConfig = createLogConfig(segmentBytes = 128 * 5) + val logConfig = LogTest.createLogConfig(segmentBytes = 128 * 5) val log = createLog(logDir, logConfig) val epoch = 0.toShort @@ -2754,7 +3222,7 @@ class LogTest { log.close() - val reloadedLogConfig = createLogConfig(segmentBytes = 1024 * 5) + val reloadedLogConfig = LogTest.createLogConfig(segmentBytes = 1024 * 5) val reloadedLog = createLog(logDir, reloadedLogConfig) val abortedTransactions = allAbortedTransactions(reloadedLog) assertEquals(List(new AbortedTxn(pid1, 0L, 29L, 8L), new AbortedTxn(pid2, 8L, 74L, 36L)), abortedTransactions) @@ -2762,7 +3230,7 @@ class LogTest { @Test def testRecoverOnlyLastSegment(): Unit = { - val logConfig = createLogConfig(segmentBytes = 128 * 5) + val logConfig = LogTest.createLogConfig(segmentBytes = 128 * 5) val log = createLog(logDir, logConfig) val epoch = 0.toShort @@ -2805,7 +3273,7 @@ class LogTest { log.close() - val reloadedLogConfig = createLogConfig(segmentBytes = 1024 * 5) + val reloadedLogConfig = LogTest.createLogConfig(segmentBytes = 1024 * 5) val reloadedLog = createLog(logDir, reloadedLogConfig, recoveryPoint = recoveryPoint) val abortedTransactions = allAbortedTransactions(reloadedLog) assertEquals(List(new AbortedTxn(pid1, 0L, 29L, 8L), new AbortedTxn(pid2, 8L, 74L, 36L)), abortedTransactions) @@ -2813,7 +3281,7 @@ class LogTest { @Test def testRecoverLastSegmentWithNoSnapshots(): Unit = { - val logConfig = createLogConfig(segmentBytes = 128 * 5) + val logConfig = LogTest.createLogConfig(segmentBytes = 128 * 5) val log = createLog(logDir, logConfig) val epoch = 0.toShort @@ -2859,7 +3327,7 @@ class LogTest { log.close() - val reloadedLogConfig = createLogConfig(segmentBytes = 1024 * 5) + val reloadedLogConfig = LogTest.createLogConfig(segmentBytes = 1024 * 5) val reloadedLog = createLog(logDir, reloadedLogConfig, recoveryPoint = recoveryPoint) val abortedTransactions = allAbortedTransactions(reloadedLog) assertEquals(List(new AbortedTxn(pid1, 0L, 29L, 8L), new AbortedTxn(pid2, 8L, 74L, 36L)), abortedTransactions) @@ -2868,7 +3336,7 @@ class LogTest { @Test def testTransactionIndexUpdatedThroughReplication(): Unit = { val epoch = 0.toShort - val logConfig = createLogConfig(segmentBytes = 1024 * 1024 * 5) + val logConfig = LogTest.createLogConfig(segmentBytes = 1024 * 1024 * 5) val log = createLog(logDir, logConfig) val buffer = ByteBuffer.allocate(2048) @@ -2914,7 +3382,7 @@ class LogTest { def testZombieCoordinatorFenced(): Unit = { val pid = 1L val epoch = 0.toShort - val logConfig = createLogConfig(segmentBytes = 1024 * 1024 * 5) + val logConfig = LogTest.createLogConfig(segmentBytes = 1024 * 1024 * 5) val log = createLog(logDir, logConfig) val append = appendTransactionalAsLeader(log, pid, epoch) @@ -2928,9 +3396,32 @@ class LogTest { appendEndTxnMarkerAsLeader(log, pid, epoch, ControlRecordType.ABORT, coordinatorEpoch = 1) } + @Test + def testZombieCoordinatorFencedEmptyTransaction(): Unit = { + val pid = 1L + val epoch = 0.toShort + val logConfig = LogTest.createLogConfig(segmentBytes = 1024 * 1024 * 5) + val log = createLog(logDir, logConfig) + + val buffer = ByteBuffer.allocate(256) + val append = appendTransactionalToBuffer(buffer, pid, epoch, leaderEpoch = 1) + append(0, 10) + appendEndTxnMarkerToBuffer(buffer, pid, epoch, 10L, ControlRecordType.COMMIT, + coordinatorEpoch = 0, leaderEpoch = 1) + + buffer.flip() + log.appendAsFollower(MemoryRecords.readableRecords(buffer)) + + appendEndTxnMarkerAsLeader(log, pid, epoch, ControlRecordType.ABORT, coordinatorEpoch = 2, leaderEpoch = 1) + appendEndTxnMarkerAsLeader(log, pid, epoch, ControlRecordType.ABORT, coordinatorEpoch = 2, leaderEpoch = 1) + assertThrows[TransactionCoordinatorFencedException] { + appendEndTxnMarkerAsLeader(log, pid, epoch, ControlRecordType.ABORT, coordinatorEpoch = 1, leaderEpoch = 1) + } + } + @Test def testFirstUnstableOffsetDoesNotExceedLogStartOffsetMidSegment(): Unit = { - val logConfig = createLogConfig(segmentBytes = 1024 * 1024 * 5) + val logConfig = LogTest.createLogConfig(segmentBytes = 1024 * 1024 * 5) val log = createLog(logDir, logConfig) val epoch = 0.toShort val pid = 1L @@ -2954,7 +3445,7 @@ class LogTest { @Test def testFirstUnstableOffsetDoesNotExceedLogStartOffsetAfterSegmentDeletion(): Unit = { - val logConfig = createLogConfig(segmentBytes = 1024 * 1024 * 5) + val logConfig = LogTest.createLogConfig(segmentBytes = 1024 * 1024 * 5) val log = createLog(logDir, logConfig) val epoch = 0.toShort val pid = 1L @@ -2979,9 +3470,50 @@ class LogTest { assertEquals(Some(8L), log.firstUnstableOffset.map(_.messageOffset)) } + @Test + def testAppendToTransactionIndexFailure(): Unit = { + val pid = 1L + val epoch = 0.toShort + val logConfig = LogTest.createLogConfig(segmentBytes = 1024 * 1024 * 5) + val log = createLog(logDir, logConfig) + + val append = appendTransactionalAsLeader(log, pid, epoch) + append(10) + + // Kind of a hack, but renaming the index to a directory ensures that the append + // to the index will fail. + log.activeSegment.txnIndex.renameTo(log.dir) + + // The append will be written to the log successfully, but the write to the index will fail + assertThrows[KafkaStorageException] { + appendEndTxnMarkerAsLeader(log, pid, epoch, ControlRecordType.ABORT, coordinatorEpoch = 1) + } + assertEquals(11L, log.logEndOffset) + assertEquals(Some(0L), log.firstUnstableOffset.map(_.messageOffset)) + + // Try the append a second time. The appended offset in the log should still increase. + assertThrows[KafkaStorageException] { + appendEndTxnMarkerAsLeader(log, pid, epoch, ControlRecordType.ABORT, coordinatorEpoch = 1) + } + assertEquals(12L, log.logEndOffset) + assertEquals(Some(0L), log.firstUnstableOffset.map(_.messageOffset)) + + // Even if the high watermark is updated, the first unstable offset does not move + log.onHighWatermarkIncremented(12L) + assertEquals(Some(0L), log.firstUnstableOffset.map(_.messageOffset)) + + log.close() + + val reopenedLog = createLog(logDir, logConfig) + assertEquals(12L, reopenedLog.logEndOffset) + assertEquals(2, reopenedLog.activeSegment.txnIndex.allAbortedTxns.size) + reopenedLog.onHighWatermarkIncremented(12L) + assertEquals(None, reopenedLog.firstUnstableOffset.map(_.messageOffset)) + } + @Test def testLastStableOffsetWithMixedProducerData() { - val logConfig = createLogConfig(segmentBytes = 1024 * 1024 * 5) + val logConfig = LogTest.createLogConfig(segmentBytes = 1024 * 1024 * 5) val log = createLog(logDir, logConfig) // for convenience, both producers share the same epoch @@ -3042,7 +3574,7 @@ class LogTest { new SimpleRecord("b".getBytes), new SimpleRecord("c".getBytes)) - val logConfig = createLogConfig(segmentBytes = records.sizeInBytes) + val logConfig = LogTest.createLogConfig(segmentBytes = records.sizeInBytes) val log = createLog(logDir, logConfig) val firstAppendInfo = log.appendAsLeader(records, leaderEpoch = 0) @@ -3073,52 +3605,14 @@ class LogTest { assertEquals(new AbortedTransaction(pid, 0), fetchDataInfo.abortedTransactions.get.head) } - def createLogConfig(segmentMs: Long = Defaults.SegmentMs, - segmentBytes: Int = Defaults.SegmentSize, - retentionMs: Long = Defaults.RetentionMs, - retentionBytes: Long = Defaults.RetentionSize, - segmentJitterMs: Long = Defaults.SegmentJitterMs, - cleanupPolicy: String = Defaults.CleanupPolicy, - maxMessageBytes: Int = Defaults.MaxMessageSize, - indexIntervalBytes: Int = Defaults.IndexInterval, - segmentIndexBytes: Int = Defaults.MaxIndexSize, - messageFormatVersion: String = Defaults.MessageFormatVersion, - fileDeleteDelayMs: Long = Defaults.FileDeleteDelayMs): LogConfig = { - val logProps = new Properties() - - logProps.put(LogConfig.SegmentMsProp, segmentMs: java.lang.Long) - logProps.put(LogConfig.SegmentBytesProp, segmentBytes: Integer) - logProps.put(LogConfig.RetentionMsProp, retentionMs: java.lang.Long) - logProps.put(LogConfig.RetentionBytesProp, retentionBytes: java.lang.Long) - logProps.put(LogConfig.SegmentJitterMsProp, segmentJitterMs: java.lang.Long) - logProps.put(LogConfig.CleanupPolicyProp, cleanupPolicy) - logProps.put(LogConfig.MaxMessageBytesProp, maxMessageBytes: Integer) - logProps.put(LogConfig.IndexIntervalBytesProp, indexIntervalBytes: Integer) - logProps.put(LogConfig.SegmentIndexBytesProp, segmentIndexBytes: Integer) - logProps.put(LogConfig.MessageFormatVersionProp, messageFormatVersion) - logProps.put(LogConfig.FileDeleteDelayMsProp, fileDeleteDelayMs: java.lang.Long) - LogConfig(logProps) - } - - def createLog(dir: File, - config: LogConfig, - logStartOffset: Long = 0L, - recoveryPoint: Long = 0L, - scheduler: Scheduler = mockTime.scheduler, - brokerTopicStats: BrokerTopicStats = brokerTopicStats, - time: Time = mockTime, - maxProducerIdExpirationMs: Int = 60 * 60 * 1000, - producerIdExpirationCheckIntervalMs: Int = LogManager.ProducerIdExpirationCheckIntervalMs): Log = { - Log(dir = dir, - config = config, - logStartOffset = logStartOffset, - recoveryPoint = recoveryPoint, - scheduler = scheduler, - brokerTopicStats = brokerTopicStats, - time = time, - maxProducerIdExpirationMs = maxProducerIdExpirationMs, - producerIdExpirationCheckIntervalMs = producerIdExpirationCheckIntervalMs, - logDirFailureChannel = new LogDirFailureChannel(10)) + @Test + def testLoadPartitionDirWithNoSegmentsShouldNotThrow() { + val dirName = Log.logDeleteDirName(new TopicPartition("foo", 3)) + val logDir = new File(tmpDir, dirName) + logDir.mkdirs() + val logConfig = LogTest.createLogConfig() + val log = createLog(logDir, logConfig) + assertEquals(1, log.numberOfSegments) } private def allAbortedTransactions(log: Log) = log.logSegments.flatMap(_.txnIndex.allAbortedTxns) @@ -3136,10 +3630,14 @@ class LogTest { } } - private def appendEndTxnMarkerAsLeader(log: Log, producerId: Long, producerEpoch: Short, - controlType: ControlRecordType, coordinatorEpoch: Int = 0): Unit = { + private def appendEndTxnMarkerAsLeader(log: Log, + producerId: Long, + producerEpoch: Short, + controlType: ControlRecordType, + coordinatorEpoch: Int = 0, + leaderEpoch: Int = 0): Unit = { val records = endTxnRecords(controlType, producerId, producerEpoch, coordinatorEpoch = coordinatorEpoch) - log.appendAsLeader(records, isFromClient = false, leaderEpoch = 0) + log.appendAsLeader(records, isFromClient = false, leaderEpoch = leaderEpoch) } private def appendNonTransactionalAsLeader(log: Log, numRecords: Int): Unit = { @@ -3150,10 +3648,14 @@ class LogTest { log.appendAsLeader(records, leaderEpoch = 0) } - private def appendTransactionalToBuffer(buffer: ByteBuffer, producerId: Long, producerEpoch: Short): (Long, Int) => Unit = { + private def appendTransactionalToBuffer(buffer: ByteBuffer, + producerId: Long, + producerEpoch: Short, + leaderEpoch: Int = 0): (Long, Int) => Unit = { var sequence = 0 (offset: Long, numRecords: Int) => { - val builder = MemoryRecords.builder(buffer, CompressionType.NONE, offset, producerId, producerEpoch, sequence, true) + val builder = MemoryRecords.builder(buffer, RecordBatch.CURRENT_MAGIC_VALUE, CompressionType.NONE, TimestampType.CREATE_TIME, + offset, System.currentTimeMillis(), producerId, producerEpoch, sequence, true, leaderEpoch) for (seq <- sequence until sequence + numRecords) { val record = new SimpleRecord(s"$seq".getBytes) builder.append(record) @@ -3164,10 +3666,15 @@ class LogTest { } } - private def appendEndTxnMarkerToBuffer(buffer: ByteBuffer, producerId: Long, producerEpoch: Short, offset: Long, - controlType: ControlRecordType, coordinatorEpoch: Int = 0): Unit = { + private def appendEndTxnMarkerToBuffer(buffer: ByteBuffer, + producerId: Long, + producerEpoch: Short, + offset: Long, + controlType: ControlRecordType, + coordinatorEpoch: Int = 0, + leaderEpoch: Int = 0): Unit = { val marker = new EndTransactionMarker(controlType, coordinatorEpoch) - MemoryRecords.writeEndTransactionalMarker(buffer, offset, mockTime.milliseconds(), 0, producerId, producerEpoch, marker) + MemoryRecords.writeEndTransactionalMarker(buffer, offset, mockTime.milliseconds(), leaderEpoch, producerId, producerEpoch, marker) } private def appendNonTransactionalToBuffer(buffer: ByteBuffer, offset: Long, numRecords: Int): Unit = { @@ -3200,4 +3707,198 @@ class LogTest { private def listProducerSnapshotOffsets: Seq[Long] = ProducerStateManager.listSnapshotFiles(logDir).map(Log.offsetFromFile).sorted + private def createLog(dir: File, + config: LogConfig, + brokerTopicStats: BrokerTopicStats = brokerTopicStats, + logStartOffset: Long = 0L, + recoveryPoint: Long = 0L, + scheduler: Scheduler = mockTime.scheduler, + time: Time = mockTime, + maxProducerIdExpirationMs: Int = 60 * 60 * 1000, + producerIdExpirationCheckIntervalMs: Int = LogManager.ProducerIdExpirationCheckIntervalMs): Log = { + LogTest.createLog(dir, config, brokerTopicStats, scheduler, time, logStartOffset, recoveryPoint, + maxProducerIdExpirationMs, producerIdExpirationCheckIntervalMs) + } + + private def createLogWithOffsetOverflow(logConfig: LogConfig): (Log, LogSegment) = { + LogTest.initializeLogDirWithOverflowedSegment(logDir) + + val log = createLog(logDir, logConfig, recoveryPoint = Long.MaxValue) + val segmentWithOverflow = LogTest.firstOverflowSegment(log).getOrElse { + Assertions.fail("Failed to create log with a segment which has overflowed offsets") + } + + (log, segmentWithOverflow) + } + + private def recoverAndCheck(config: LogConfig, + expectedKeys: Iterable[Long], + expectDeletedFiles: Boolean = true): Log = { + LogTest.recoverAndCheck(logDir, config, expectedKeys, brokerTopicStats, mockTime, mockTime.scheduler, + expectDeletedFiles) + } + + private def readLog(log: Log, startOffset: Long, maxLength: Int, + maxOffset: Option[Long] = None, + minOneMessage: Boolean = true): FetchDataInfo = { + log.read(startOffset, maxLength, maxOffset, minOneMessage, isolationLevel = IsolationLevel.READ_UNCOMMITTED) + } +} + +object LogTest { + def createLogConfig(segmentMs: Long = Defaults.SegmentMs, + segmentBytes: Int = Defaults.SegmentSize, + retentionMs: Long = Defaults.RetentionMs, + retentionBytes: Long = Defaults.RetentionSize, + segmentJitterMs: Long = Defaults.SegmentJitterMs, + cleanupPolicy: String = Defaults.CleanupPolicy, + maxMessageBytes: Int = Defaults.MaxMessageSize, + indexIntervalBytes: Int = Defaults.IndexInterval, + segmentIndexBytes: Int = Defaults.MaxIndexSize, + messageFormatVersion: String = Defaults.MessageFormatVersion, + fileDeleteDelayMs: Long = Defaults.FileDeleteDelayMs): LogConfig = { + val logProps = new Properties() + + logProps.put(LogConfig.SegmentMsProp, segmentMs: java.lang.Long) + logProps.put(LogConfig.SegmentBytesProp, segmentBytes: Integer) + logProps.put(LogConfig.RetentionMsProp, retentionMs: java.lang.Long) + logProps.put(LogConfig.RetentionBytesProp, retentionBytes: java.lang.Long) + logProps.put(LogConfig.SegmentJitterMsProp, segmentJitterMs: java.lang.Long) + logProps.put(LogConfig.CleanupPolicyProp, cleanupPolicy) + logProps.put(LogConfig.MaxMessageBytesProp, maxMessageBytes: Integer) + logProps.put(LogConfig.IndexIntervalBytesProp, indexIntervalBytes: Integer) + logProps.put(LogConfig.SegmentIndexBytesProp, segmentIndexBytes: Integer) + logProps.put(LogConfig.MessageFormatVersionProp, messageFormatVersion) + logProps.put(LogConfig.FileDeleteDelayMsProp, fileDeleteDelayMs: java.lang.Long) + LogConfig(logProps) + } + + def createLog(dir: File, + config: LogConfig, + brokerTopicStats: BrokerTopicStats, + scheduler: Scheduler, + time: Time, + logStartOffset: Long = 0L, + recoveryPoint: Long = 0L, + maxProducerIdExpirationMs: Int = 60 * 60 * 1000, + producerIdExpirationCheckIntervalMs: Int = LogManager.ProducerIdExpirationCheckIntervalMs): Log = { + Log(dir = dir, + config = config, + logStartOffset = logStartOffset, + recoveryPoint = recoveryPoint, + scheduler = scheduler, + brokerTopicStats = brokerTopicStats, + time = time, + maxProducerIdExpirationMs = maxProducerIdExpirationMs, + producerIdExpirationCheckIntervalMs = producerIdExpirationCheckIntervalMs, + logDirFailureChannel = new LogDirFailureChannel(10)) + } + + /** + * Check if the given log contains any segment with records that cause offset overflow. + * @param log Log to check + * @return true if log contains at least one segment with offset overflow; false otherwise + */ + def hasOffsetOverflow(log: Log): Boolean = firstOverflowSegment(log).isDefined + + def firstOverflowSegment(log: Log): Option[LogSegment] = { + def hasOverflow(baseOffset: Long, batch: RecordBatch): Boolean = + batch.lastOffset > baseOffset + Int.MaxValue || batch.baseOffset < baseOffset + + for (segment <- log.logSegments) { + val overflowBatch = segment.log.batches.asScala.find(batch => hasOverflow(segment.baseOffset, batch)) + if (overflowBatch.isDefined) + return Some(segment) + } + None + } + + private def rawSegment(logDir: File, baseOffset: Long): FileRecords = + FileRecords.open(Log.logFile(logDir, baseOffset)) + + /** + * Initialize the given log directory with a set of segments, one of which will have an + * offset which overflows the segment + */ + def initializeLogDirWithOverflowedSegment(logDir: File): Unit = { + def writeSampleBatches(baseOffset: Long, segment: FileRecords): Long = { + def record(offset: Long) = { + val data = offset.toString.getBytes + new SimpleRecord(data, data) + } + + segment.append(MemoryRecords.withRecords(baseOffset, CompressionType.NONE, 0, + record(baseOffset))) + segment.append(MemoryRecords.withRecords(baseOffset + 1, CompressionType.NONE, 0, + record(baseOffset + 1), + record(baseOffset + 2))) + segment.append(MemoryRecords.withRecords(baseOffset + Int.MaxValue - 1, CompressionType.NONE, 0, + record(baseOffset + Int.MaxValue - 1))) + baseOffset + Int.MaxValue + } + + def writeNormalSegment(baseOffset: Long): Long = { + val segment = rawSegment(logDir, baseOffset) + try writeSampleBatches(baseOffset, segment) + finally segment.close() + } + + def writeOverflowSegment(baseOffset: Long): Long = { + val segment = rawSegment(logDir, baseOffset) + try { + val nextOffset = writeSampleBatches(baseOffset, segment) + writeSampleBatches(nextOffset, segment) + } finally segment.close() + } + + // We create three segments, the second of which contains offsets which overflow + var nextOffset = 0L + nextOffset = writeNormalSegment(nextOffset) + nextOffset = writeOverflowSegment(nextOffset) + writeNormalSegment(nextOffset) + } + + def allRecords(log: Log): List[Record] = { + val recordsFound = ListBuffer[Record]() + for (logSegment <- log.logSegments) { + for (batch <- logSegment.log.batches.asScala) { + recordsFound ++= batch.iterator().asScala + } + } + recordsFound.toList + } + + def verifyRecordsInLog(log: Log, expectedRecords: List[Record]): Unit = { + assertEquals(expectedRecords, allRecords(log)) + } + + /* extract all the keys from a log */ + def keysInLog(log: Log): Iterable[Long] = { + for (logSegment <- log.logSegments; + batch <- logSegment.log.batches.asScala if !batch.isControlBatch; + record <- batch.asScala if record.hasValue && record.hasKey) + yield TestUtils.readString(record.key).toLong + } + + def recoverAndCheck(logDir: File, + config: LogConfig, + expectedKeys: Iterable[Long], + brokerTopicStats: BrokerTopicStats, + time: Time, + scheduler: Scheduler, + expectDeletedFiles: Boolean = false): Log = { + // Recover log file and check that after recovery, keys are as expected + // and all temporary files have been deleted + val recoveredLog = createLog(logDir, config, brokerTopicStats, scheduler, time) + time.sleep(config.fileDeleteDelayMs + 1) + for (file <- logDir.listFiles) { + if (!expectDeletedFiles) + 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, LogTest.keysInLog(recoveredLog)) + assertFalse(LogTest.hasOffsetOverflow(recoveredLog)) + recoveredLog + } } diff --git a/core/src/test/scala/unit/kafka/log/OffsetIndexTest.scala b/core/src/test/scala/unit/kafka/log/OffsetIndexTest.scala index 8fa3cc196481a..f47da995dd5f9 100644 --- a/core/src/test/scala/unit/kafka/log/OffsetIndexTest.scala +++ b/core/src/test/scala/unit/kafka/log/OffsetIndexTest.scala @@ -29,16 +29,17 @@ import org.scalatest.junit.JUnitSuite import scala.collection._ import scala.util.Random import kafka.utils.TestUtils -import kafka.common.InvalidOffsetException +import org.apache.kafka.common.errors.InvalidOffsetException class OffsetIndexTest extends JUnitSuite { var idx: OffsetIndex = null val maxEntries = 30 + val baseOffset = 45L @Before def setup() { - this.idx = new OffsetIndex(nonExistentTempFile(), baseOffset = 45L, maxIndexSize = 30 * 8) + this.idx = new OffsetIndex(nonExistentTempFile(), baseOffset, maxIndexSize = 30 * 8) } @After @@ -102,10 +103,10 @@ class OffsetIndexTest extends JUnitSuite { @Test def testFetchUpperBoundOffset() { - val first = OffsetPosition(0, 0) - val second = OffsetPosition(1, 10) - val third = OffsetPosition(2, 23) - val fourth = OffsetPosition(3, 37) + val first = OffsetPosition(baseOffset + 0, 0) + val second = OffsetPosition(baseOffset + 1, 10) + val third = OffsetPosition(baseOffset + 2, 23) + val fourth = OffsetPosition(baseOffset + 3, 37) assertEquals(None, idx.fetchUpperBoundOffset(first, 5)) @@ -177,6 +178,15 @@ class OffsetIndexTest extends JUnitSuite { // mmap should be null after unmap causing lookup to throw a NPE intercept[NullPointerException](idx.lookup(1)) } + + @Test + def testSanityLastOffsetEqualToBaseOffset(): Unit = { + // Test index sanity for the case where the last offset appended to the index is equal to the base offset + val baseOffset = 20L + val idx = new OffsetIndex(nonExistentTempFile(), baseOffset = baseOffset, maxIndexSize = 10 * 8) + idx.append(baseOffset, 0) + idx.sanityCheck() + } def assertWriteFails[T](message: String, idx: OffsetIndex, offset: Int, klass: Class[T]) { try { diff --git a/core/src/test/scala/unit/kafka/log/ProducerStateManagerTest.scala b/core/src/test/scala/unit/kafka/log/ProducerStateManagerTest.scala index 053aed7c91535..f09e5eeafddcf 100644 --- a/core/src/test/scala/unit/kafka/log/ProducerStateManagerTest.scala +++ b/core/src/test/scala/unit/kafka/log/ProducerStateManagerTest.scala @@ -21,14 +21,16 @@ import java.io.File import java.nio.ByteBuffer import java.nio.channels.FileChannel import java.nio.file.StandardOpenOption +import java.util.Collections import kafka.server.LogOffsetMetadata import kafka.utils.TestUtils import org.apache.kafka.common.TopicPartition import org.apache.kafka.common.errors._ import org.apache.kafka.common.internals.Topic -import org.apache.kafka.common.record.{ControlRecordType, EndTransactionMarker, RecordBatch} +import org.apache.kafka.common.record._ import org.apache.kafka.common.utils.{MockTime, Utils} +import org.easymock.EasyMock import org.junit.Assert._ import org.junit.{After, Before, Test} import org.scalatest.junit.JUnitSuite @@ -81,6 +83,35 @@ class ProducerStateManagerTest extends JUnitSuite { } } + @Test + def testAppendTxnMarkerWithNoProducerState(): Unit = { + val producerEpoch = 2.toShort + appendEndTxnMarker(stateManager, producerId, producerEpoch, ControlRecordType.COMMIT, offset = 27L) + + val firstEntry = stateManager.lastEntry(producerId).getOrElse(fail("Expected last entry to be defined")) + assertEquals(producerEpoch, firstEntry.producerEpoch) + assertEquals(producerId, firstEntry.producerId) + assertEquals(RecordBatch.NO_SEQUENCE, firstEntry.lastSeq) + + // Fencing should continue to work even if the marker is the only thing left + assertThrows[ProducerFencedException] { + append(stateManager, producerId, 0.toShort, 0, 0L, 4L) + } + + // If the transaction marker is the only thing left in the log, then an attempt to write using a + // non-zero sequence number should cause an UnknownProducerId, so that the producer can reset its state + assertThrows[UnknownProducerIdException] { + append(stateManager, producerId, producerEpoch, 17, 0L, 4L) + } + + // The broker should accept the request if the sequence number is reset to 0 + append(stateManager, producerId, producerEpoch, 0, 39L, 4L) + val secondEntry = stateManager.lastEntry(producerId).getOrElse(fail("Expected last entry to be defined")) + assertEquals(producerEpoch, secondEntry.producerEpoch) + assertEquals(producerId, secondEntry.producerId) + assertEquals(0, secondEntry.lastSeq) + } + @Test def testProducerSequenceWrapAround(): Unit = { val epoch = 15.toShort @@ -100,6 +131,24 @@ class ProducerStateManagerTest extends JUnitSuite { assertEquals(0, lastEntry.lastSeq) } + @Test + def testProducerSequenceWithWrapAroundBatchRecord(): Unit = { + val epoch = 15.toShort + + val appendInfo = stateManager.prepareUpdate(producerId, isFromClient = false) + // Sequence number wrap around + appendInfo.append(epoch, Int.MaxValue-10, 9, time.milliseconds(), 2000L, 2020L, isTransactional = false) + assertEquals(None, stateManager.lastEntry(producerId)) + stateManager.update(appendInfo) + assertTrue(stateManager.lastEntry(producerId).isDefined) + + val lastEntry = stateManager.lastEntry(producerId).get + assertEquals(Int.MaxValue-10, lastEntry.firstSeq) + assertEquals(9, lastEntry.lastSeq) + assertEquals(2000L, lastEntry.firstOffset) + assertEquals(2020L, lastEntry.lastDataOffset) + } + @Test(expected = classOf[OutOfOrderSequenceException]) def testProducerSequenceInvalidWrapAround(): Unit = { val epoch = 15.toShort @@ -159,8 +208,8 @@ class ProducerStateManagerTest extends JUnitSuite { val producerEpoch = 0.toShort val offset = 992342L val seq = 0 - val producerAppendInfo = new ProducerAppendInfo(producerId, ProducerStateEntry.empty(producerId), ValidationType.Full) - producerAppendInfo.append(producerEpoch, seq, seq, time.milliseconds(), offset, isTransactional = true) + val producerAppendInfo = new ProducerAppendInfo(partition, producerId, ProducerStateEntry.empty(producerId), ValidationType.Full) + producerAppendInfo.append(producerEpoch, seq, seq, time.milliseconds(), offset, offset, isTransactional = true) val logOffsetMetadata = new LogOffsetMetadata(messageOffset = offset, segmentBaseOffset = 990000L, relativePositionInSegment = 234224) @@ -170,13 +219,66 @@ class ProducerStateManagerTest extends JUnitSuite { assertEquals(Some(logOffsetMetadata), stateManager.firstUnstableOffset) } + @Test + def testLastStableOffsetCompletedTxn(): Unit = { + val producerEpoch = 0.toShort + val segmentBaseOffset = 990000L + + def beginTxn(producerId: Long, startOffset: Long): Unit = { + val relativeOffset = (startOffset - segmentBaseOffset).toInt + val producerAppendInfo = new ProducerAppendInfo( + partition, + producerId, + ProducerStateEntry.empty(producerId), + ValidationType.Full) + producerAppendInfo.append(producerEpoch, 0, 0, time.milliseconds(), startOffset, startOffset, isTransactional = true) + val logOffsetMetadata = LogOffsetMetadata(messageOffset = startOffset, segmentBaseOffset = segmentBaseOffset, + relativePositionInSegment = 50 * relativeOffset) + producerAppendInfo.maybeCacheTxnFirstOffsetMetadata(logOffsetMetadata) + stateManager.update(producerAppendInfo) + } + + val producerId1 = producerId + val startOffset1 = 992342L + beginTxn(producerId1, startOffset1) + + val producerId2 = producerId + 1 + val startOffset2 = startOffset1 + 25 + beginTxn(producerId2, startOffset2) + + val producerId3 = producerId + 2 + val startOffset3 = startOffset1 + 57 + beginTxn(producerId3, startOffset3) + + val lastOffset1 = startOffset3 + 15 + val completedTxn1 = CompletedTxn(producerId1, startOffset1, lastOffset1, isAborted = false) + assertEquals(startOffset2, stateManager.lastStableOffset(completedTxn1)) + stateManager.completeTxn(completedTxn1) + stateManager.onHighWatermarkUpdated(lastOffset1 + 1) + assertEquals(Some(startOffset2), stateManager.firstUnstableOffset.map(_.messageOffset)) + + val lastOffset3 = lastOffset1 + 20 + val completedTxn3 = CompletedTxn(producerId3, startOffset3, lastOffset3, isAborted = false) + assertEquals(startOffset2, stateManager.lastStableOffset(completedTxn3)) + stateManager.completeTxn(completedTxn3) + stateManager.onHighWatermarkUpdated(lastOffset3 + 1) + assertEquals(Some(startOffset2), stateManager.firstUnstableOffset.map(_.messageOffset)) + + val lastOffset2 = lastOffset3 + 78 + val completedTxn2 = CompletedTxn(producerId2, startOffset2, lastOffset2, isAborted = false) + assertEquals(lastOffset2 + 1, stateManager.lastStableOffset(completedTxn2)) + stateManager.completeTxn(completedTxn2) + stateManager.onHighWatermarkUpdated(lastOffset2 + 1) + assertEquals(None, stateManager.firstUnstableOffset) + } + @Test def testNonMatchingTxnFirstOffsetMetadataNotCached(): Unit = { val producerEpoch = 0.toShort val offset = 992342L val seq = 0 - val producerAppendInfo = new ProducerAppendInfo(producerId, ProducerStateEntry.empty(producerId), ValidationType.Full) - producerAppendInfo.append(producerEpoch, seq, seq, time.milliseconds(), offset, isTransactional = true) + val producerAppendInfo = new ProducerAppendInfo(partition, producerId, ProducerStateEntry.empty(producerId), ValidationType.Full) + producerAppendInfo.append(producerEpoch, seq, seq, time.milliseconds(), offset, offset, isTransactional = true) // use some other offset to simulate a follower append where the log offset metadata won't typically // match any of the transaction first offsets @@ -193,13 +295,13 @@ class ProducerStateManagerTest extends JUnitSuite { val producerEpoch = 0.toShort val appendInfo = stateManager.prepareUpdate(producerId, isFromClient = true) - appendInfo.append(producerEpoch, 0, 5, time.milliseconds(), 20L, isTransactional = false) + appendInfo.append(producerEpoch, 0, 5, time.milliseconds(), 15L, 20L, isTransactional = false) assertEquals(None, stateManager.lastEntry(producerId)) stateManager.update(appendInfo) assertTrue(stateManager.lastEntry(producerId).isDefined) val nextAppendInfo = stateManager.prepareUpdate(producerId, isFromClient = true) - nextAppendInfo.append(producerEpoch, 6, 10, time.milliseconds(), 30L, isTransactional = false) + nextAppendInfo.append(producerEpoch, 6, 10, time.milliseconds(), 26L, 30L, isTransactional = false) assertTrue(stateManager.lastEntry(producerId).isDefined) var lastEntry = stateManager.lastEntry(producerId).get @@ -222,7 +324,7 @@ class ProducerStateManagerTest extends JUnitSuite { append(stateManager, producerId, producerEpoch, 0, offset) val appendInfo = stateManager.prepareUpdate(producerId, isFromClient = true) - appendInfo.append(producerEpoch, 1, 5, time.milliseconds(), 20L, isTransactional = true) + appendInfo.append(producerEpoch, 1, 5, time.milliseconds(), 16L, 20L, isTransactional = true) var lastEntry = appendInfo.toEntry assertEquals(producerEpoch, lastEntry.producerEpoch) assertEquals(1, lastEntry.firstSeq) @@ -232,7 +334,7 @@ class ProducerStateManagerTest extends JUnitSuite { assertEquals(Some(16L), lastEntry.currentTxnFirstOffset) assertEquals(List(new TxnMetadata(producerId, 16L)), appendInfo.startedTransactions) - appendInfo.append(producerEpoch, 6, 10, time.milliseconds(), 30L, isTransactional = true) + appendInfo.append(producerEpoch, 6, 10, time.milliseconds(), 26L, 30L, isTransactional = true) lastEntry = appendInfo.toEntry assertEquals(producerEpoch, lastEntry.producerEpoch) assertEquals(1, lastEntry.firstSeq) @@ -717,6 +819,22 @@ class ProducerStateManagerTest extends JUnitSuite { } } + @Test + def testAppendEmptyControlBatch(): Unit = { + val producerId = 23423L + val producerEpoch = 145.toShort + val baseOffset = 15 + + val batch = EasyMock.createMock(classOf[RecordBatch]) + EasyMock.expect(batch.isControlBatch).andReturn(true).once + EasyMock.expect(batch.iterator).andReturn(Collections.emptyIterator[Record]).once + EasyMock.replay(batch) + + // Appending the empty control batch should not throw and a new transaction shouldn't be started + append(stateManager, producerId, producerEpoch, baseOffset, batch, isFromClient = true) + assertEquals(None, stateManager.lastEntry(producerId).get.currentTxnFirstOffset) + } + private def testLoadFromCorruptSnapshot(makeFileCorrupt: FileChannel => Unit): Unit = { val epoch = 0.toShort val producerId = 1L @@ -758,7 +876,8 @@ class ProducerStateManagerTest extends JUnitSuite { val endTxnMarker = new EndTransactionMarker(controlType, coordinatorEpoch) val completedTxn = producerAppendInfo.appendEndTxnMarker(endTxnMarker, producerEpoch, offset, timestamp) mapping.update(producerAppendInfo) - val lastStableOffset = mapping.completeTxn(completedTxn) + val lastStableOffset = mapping.lastStableOffset(completedTxn) + mapping.completeTxn(completedTxn) mapping.updateMapEndOffset(offset + 1) (completedTxn, lastStableOffset) } @@ -772,7 +891,19 @@ class ProducerStateManagerTest extends JUnitSuite { isTransactional: Boolean = false, isFromClient : Boolean = true): Unit = { val producerAppendInfo = stateManager.prepareUpdate(producerId, isFromClient) - producerAppendInfo.append(producerEpoch, seq, seq, timestamp, offset, isTransactional) + producerAppendInfo.append(producerEpoch, seq, seq, timestamp, offset, offset, isTransactional) + stateManager.update(producerAppendInfo) + stateManager.updateMapEndOffset(offset + 1) + } + + private def append(stateManager: ProducerStateManager, + producerId: Long, + producerEpoch: Short, + offset: Long, + batch: RecordBatch, + isFromClient : Boolean): Unit = { + val producerAppendInfo = stateManager.prepareUpdate(producerId, isFromClient) + producerAppendInfo.append(batch) stateManager.update(producerAppendInfo) stateManager.updateMapEndOffset(offset + 1) } diff --git a/core/src/test/scala/unit/kafka/log/TimeIndexTest.scala b/core/src/test/scala/unit/kafka/log/TimeIndexTest.scala index 8520f8917ec35..b9478cdcc23ac 100644 --- a/core/src/test/scala/unit/kafka/log/TimeIndexTest.scala +++ b/core/src/test/scala/unit/kafka/log/TimeIndexTest.scala @@ -19,10 +19,10 @@ package kafka.log import java.io.File -import kafka.common.InvalidOffsetException import kafka.utils.TestUtils -import org.junit.{Test, After, Before} -import org.junit.Assert.{assertEquals} +import org.apache.kafka.common.errors.InvalidOffsetException +import org.junit.{After, Before, Test} +import org.junit.Assert.assertEquals import org.scalatest.junit.JUnitSuite /** diff --git a/core/src/test/scala/unit/kafka/message/BaseMessageSetTestCases.scala b/core/src/test/scala/unit/kafka/message/BaseMessageSetTestCases.scala deleted file mode 100644 index 40581ed20686b..0000000000000 --- a/core/src/test/scala/unit/kafka/message/BaseMessageSetTestCases.scala +++ /dev/null @@ -1,133 +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.message - -import java.nio.ByteBuffer -import java.nio.channels.{FileChannel, GatheringByteChannel} -import java.nio.file.StandardOpenOption - -import org.junit.Assert._ -import kafka.utils.TestUtils._ -import org.apache.kafka.common.record.FileRecords -import org.scalatest.junit.JUnitSuite -import org.junit.Test - -import scala.collection.mutable.ArrayBuffer -import scala.collection.JavaConverters._ - -trait BaseMessageSetTestCases extends JUnitSuite { - - private class StubByteChannel(bytesToConsumePerBuffer: Int) extends GatheringByteChannel { - - val data = new ArrayBuffer[Byte] - - def write(srcs: Array[ByteBuffer], offset: Int, length: Int): Long = { - srcs.map { src => - val array = new Array[Byte](math.min(bytesToConsumePerBuffer, src.remaining)) - src.get(array) - data ++= array - array.length - }.sum - } - - def write(srcs: Array[ByteBuffer]): Long = write(srcs, 0, srcs.map(_.remaining).sum) - - def write(src: ByteBuffer): Int = write(Array(src)).toInt - - def isOpen: Boolean = true - - def close() {} - - } - - - val messages = Array(new Message("abcd".getBytes), new Message("efgh".getBytes), new Message("ijkl".getBytes)) - - def createMessageSet(messages: Seq[Message]): MessageSet - - @Test - def testWrittenEqualsRead() { - val messageSet = createMessageSet(messages) - assertEquals(messages.toVector, messageSet.toVector.map(m => m.message)) - } - - @Test - def testIteratorIsConsistent() { - val m = createMessageSet(messages) - // two iterators over the same set should give the same results - checkEquals(m.iterator, m.iterator) - } - - @Test - def testSizeInBytes() { - assertEquals("Empty message set should have 0 bytes.", - 0, - createMessageSet(Array[Message]()).sizeInBytes) - assertEquals("Predicted size should equal actual size.", - MessageSet.messageSetSize(messages), - createMessageSet(messages).sizeInBytes) - } - - @Test - def testWriteTo() { - // test empty message set - checkWriteToWithMessageSet(createMessageSet(Array[Message]())) - checkWriteToWithMessageSet(createMessageSet(messages)) - } - - /* Tests that writing to a channel that doesn't consume all the bytes in the buffer works correctly */ - @Test - def testWriteToChannelThatConsumesPartially() { - val bytesToConsumePerBuffer = 50 - val messages = (0 until 10).map(_ => new Message(randomString(100).getBytes)) - val messageSet = createMessageSet(messages) - val messageSetSize = messageSet.sizeInBytes - - val channel = new StubByteChannel(bytesToConsumePerBuffer) - - var remaining = messageSetSize - var iterations = 0 - while (remaining > 0) { - remaining -= messageSet.asRecords.writeTo(channel, messageSetSize - remaining, remaining).toInt - iterations += 1 - } - - assertEquals((messageSetSize / bytesToConsumePerBuffer) + 1, iterations) - checkEquals(new ByteBufferMessageSet(ByteBuffer.wrap(channel.data.toArray)).iterator, messageSet.iterator) - } - - def checkWriteToWithMessageSet(messageSet: MessageSet) { - checkWriteWithMessageSet(messageSet, messageSet.asRecords.writeTo(_, 0, messageSet.sizeInBytes)) - } - - def checkWriteWithMessageSet(set: MessageSet, write: GatheringByteChannel => Long) { - // do the write twice to ensure the message set is restored to its original state - for (_ <- 0 to 1) { - val file = tempFile() - val channel = FileChannel.open(file.toPath, StandardOpenOption.READ, StandardOpenOption.WRITE) - try { - val written = write(channel) - assertEquals("Expect to write the number of bytes in the set.", set.sizeInBytes, written) - val fileRecords = new FileRecords(file, channel, 0, Integer.MAX_VALUE, false) - assertEquals(set.asRecords.records.asScala.toVector, fileRecords.records.asScala.toVector) - checkEquals(set.asRecords.records.iterator, fileRecords.records.iterator) - } finally channel.close() - } - } - -} diff --git a/core/src/test/scala/unit/kafka/message/ByteBufferMessageSetTest.scala b/core/src/test/scala/unit/kafka/message/ByteBufferMessageSetTest.scala deleted file mode 100644 index 00f9dc9f396ab..0000000000000 --- a/core/src/test/scala/unit/kafka/message/ByteBufferMessageSetTest.scala +++ /dev/null @@ -1,167 +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.message - -import java.nio._ - -import kafka.utils.TestUtils -import org.junit.Assert._ -import org.junit.Test - -class ByteBufferMessageSetTest extends BaseMessageSetTestCases { - - override def createMessageSet(messages: Seq[Message]): ByteBufferMessageSet = - new ByteBufferMessageSet(NoCompressionCodec, messages: _*) - - @Test - def testValidBytes() { - { - val messages = new ByteBufferMessageSet(NoCompressionCodec, new Message("hello".getBytes), new Message("there".getBytes)) - val buffer = ByteBuffer.allocate(messages.sizeInBytes + 2) - buffer.put(messages.buffer) - buffer.putShort(4) - val messagesPlus = new ByteBufferMessageSet(buffer) - assertEquals("Adding invalid bytes shouldn't change byte count", messages.validBytes, messagesPlus.validBytes) - } - - // test valid bytes on empty ByteBufferMessageSet - { - assertEquals("Valid bytes on an empty ByteBufferMessageSet should return 0", 0, - MessageSet.Empty.asInstanceOf[ByteBufferMessageSet].validBytes) - } - } - - @Test - def testValidBytesWithCompression() { - val messages = new ByteBufferMessageSet(DefaultCompressionCodec, new Message("hello".getBytes), new Message("there".getBytes)) - val buffer = ByteBuffer.allocate(messages.sizeInBytes + 2) - buffer.put(messages.buffer) - buffer.putShort(4) - val messagesPlus = new ByteBufferMessageSet(buffer) - assertEquals("Adding invalid bytes shouldn't change byte count", messages.validBytes, messagesPlus.validBytes) - } - - @Test - def testEquals() { - var messages = new ByteBufferMessageSet(DefaultCompressionCodec, new Message("hello".getBytes), new Message("there".getBytes)) - var moreMessages = new ByteBufferMessageSet(DefaultCompressionCodec, new Message("hello".getBytes), new Message("there".getBytes)) - - assertTrue(messages.equals(moreMessages)) - - messages = new ByteBufferMessageSet(NoCompressionCodec, new Message("hello".getBytes), new Message("there".getBytes)) - moreMessages = new ByteBufferMessageSet(NoCompressionCodec, new Message("hello".getBytes), new Message("there".getBytes)) - - assertTrue(messages.equals(moreMessages)) - } - - - @Test - def testIterator() { - val messageList = List( - new Message("msg1".getBytes), - new Message("msg2".getBytes), - new Message("msg3".getBytes) - ) - - // test for uncompressed regular messages - { - val messageSet = new ByteBufferMessageSet(NoCompressionCodec, messageList: _*) - TestUtils.checkEquals[Message](messageList.iterator, TestUtils.getMessageIterator(messageSet.iterator)) - //make sure ByteBufferMessageSet is re-iterable. - TestUtils.checkEquals[Message](messageList.iterator, TestUtils.getMessageIterator(messageSet.iterator)) - - //make sure shallow iterator is the same as deep iterator - TestUtils.checkEquals[Message](TestUtils.getMessageIterator(messageSet.shallowIterator), - TestUtils.getMessageIterator(messageSet.iterator)) - } - - // test for compressed regular messages - { - val messageSet = new ByteBufferMessageSet(DefaultCompressionCodec, messageList: _*) - TestUtils.checkEquals[Message](messageList.iterator, TestUtils.getMessageIterator(messageSet.iterator)) - //make sure ByteBufferMessageSet is re-iterable. - TestUtils.checkEquals[Message](messageList.iterator, TestUtils.getMessageIterator(messageSet.iterator)) - verifyShallowIterator(messageSet) - } - - // test for mixed empty and non-empty messagesets uncompressed - { - val emptyMessageList : List[Message] = Nil - val emptyMessageSet = new ByteBufferMessageSet(NoCompressionCodec, emptyMessageList: _*) - val regularMessgeSet = new ByteBufferMessageSet(NoCompressionCodec, messageList: _*) - val buffer = ByteBuffer.allocate(emptyMessageSet.buffer.limit() + regularMessgeSet.buffer.limit()) - buffer.put(emptyMessageSet.buffer) - buffer.put(regularMessgeSet.buffer) - buffer.rewind - val mixedMessageSet = new ByteBufferMessageSet(buffer) - TestUtils.checkEquals[Message](messageList.iterator, TestUtils.getMessageIterator(mixedMessageSet.iterator)) - //make sure ByteBufferMessageSet is re-iterable. - TestUtils.checkEquals[Message](messageList.iterator, TestUtils.getMessageIterator(mixedMessageSet.iterator)) - //make sure shallow iterator is the same as deep iterator - TestUtils.checkEquals[Message](TestUtils.getMessageIterator(mixedMessageSet.shallowIterator), - TestUtils.getMessageIterator(mixedMessageSet.iterator)) - } - - // test for mixed empty and non-empty messagesets compressed - { - val emptyMessageList : List[Message] = Nil - val emptyMessageSet = new ByteBufferMessageSet(DefaultCompressionCodec, emptyMessageList: _*) - val regularMessgeSet = new ByteBufferMessageSet(DefaultCompressionCodec, messageList: _*) - val buffer = ByteBuffer.allocate(emptyMessageSet.buffer.limit() + regularMessgeSet.buffer.limit()) - buffer.put(emptyMessageSet.buffer) - buffer.put(regularMessgeSet.buffer) - buffer.rewind - val mixedMessageSet = new ByteBufferMessageSet(buffer) - TestUtils.checkEquals[Message](messageList.iterator, TestUtils.getMessageIterator(mixedMessageSet.iterator)) - //make sure ByteBufferMessageSet is re-iterable. - TestUtils.checkEquals[Message](messageList.iterator, TestUtils.getMessageIterator(mixedMessageSet.iterator)) - verifyShallowIterator(mixedMessageSet) - } - } - - @Test - def testMessageWithProvidedOffsetSeq() { - val offsets = Seq(0L, 2L) - val messages = new ByteBufferMessageSet( - compressionCodec = NoCompressionCodec, - offsetSeq = offsets, - new Message("hello".getBytes), - new Message("goodbye".getBytes)) - val iter = messages.iterator - assertEquals("first offset should be 0", 0L, iter.next().offset) - assertEquals("second offset should be 2", 2L, iter.next().offset) - } - - /* check that offsets are assigned based on byte offset from the given base offset */ - def checkOffsets(messages: ByteBufferMessageSet, baseOffset: Long) { - assertTrue("Message set should not be empty", messages.nonEmpty) - var offset = baseOffset - for(entry <- messages) { - assertEquals("Unexpected offset in message set iterator", offset, entry.offset) - offset += 1 - } - } - - def verifyShallowIterator(messageSet: ByteBufferMessageSet) { - //make sure the offsets returned by a shallow iterator is a subset of that of a deep iterator - val shallowOffsets = messageSet.shallowIterator.map(msgAndOff => msgAndOff.offset).toSet - val deepOffsets = messageSet.iterator.map(msgAndOff => msgAndOff.offset).toSet - assertTrue(shallowOffsets.subsetOf(deepOffsets)) - } - -} diff --git a/core/src/test/scala/unit/kafka/message/MessageCompressionTest.scala b/core/src/test/scala/unit/kafka/message/MessageCompressionTest.scala deleted file mode 100644 index ab8046531cf9e..0000000000000 --- a/core/src/test/scala/unit/kafka/message/MessageCompressionTest.scala +++ /dev/null @@ -1,87 +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.message - -import java.io.ByteArrayOutputStream -import scala.collection._ -import org.junit._ -import org.junit.Assert._ - -class MessageCompressionTest { - - @Test - def testSimpleCompressDecompress() { - val codecs = mutable.ArrayBuffer[CompressionCodec](GZIPCompressionCodec) - if (isSnappyAvailable) - codecs += SnappyCompressionCodec - if (isLZ4Available) - codecs += LZ4CompressionCodec - for (codec <- codecs) - testSimpleCompressDecompress(codec) - } - - // A quick test to ensure any growth or increase in compression size is known when upgrading libraries - @Test - def testCompressSize() { - val bytes1k: Array[Byte] = (0 until 1000).map(_.toByte).toArray - val bytes2k: Array[Byte] = (1000 until 2000).map(_.toByte).toArray - val bytes3k: Array[Byte] = (3000 until 4000).map(_.toByte).toArray - val messages: List[Message] = List(new Message(bytes1k, Message.NoTimestamp, Message.MagicValue_V1), - new Message(bytes2k, Message.NoTimestamp, Message.MagicValue_V1), - new Message(bytes3k, Message.NoTimestamp, Message.MagicValue_V1)) - - testCompressSize(GZIPCompressionCodec, messages, 396) - - if (isSnappyAvailable) - testCompressSize(SnappyCompressionCodec, messages, 503) - - if (isLZ4Available) - testCompressSize(LZ4CompressionCodec, messages, 387) - } - - def testSimpleCompressDecompress(compressionCodec: CompressionCodec) { - val messages = List[Message](new Message("hi there".getBytes), new Message("I am fine".getBytes), new Message("I am not so well today".getBytes)) - val messageSet = new ByteBufferMessageSet(compressionCodec = compressionCodec, messages = messages:_*) - assertEquals(compressionCodec, messageSet.shallowIterator.next().message.compressionCodec) - val decompressed = messageSet.iterator.map(_.message).toList - assertEquals(messages, decompressed) - } - - def testCompressSize(compressionCodec: CompressionCodec, messages: List[Message], expectedSize: Int) { - val messageSet = new ByteBufferMessageSet(compressionCodec = compressionCodec, messages = messages:_*) - assertEquals(s"$compressionCodec size has changed.", expectedSize, messageSet.sizeInBytes) - } - - def isSnappyAvailable: Boolean = { - try { - new org.xerial.snappy.SnappyOutputStream(new ByteArrayOutputStream()) - true - } catch { - case _: UnsatisfiedLinkError | _: org.xerial.snappy.SnappyError => false - } - } - - def isLZ4Available: Boolean = { - try { - new net.jpountz.lz4.LZ4BlockOutputStream(new ByteArrayOutputStream()) - true - } catch { - case _: UnsatisfiedLinkError => false - } - } -} diff --git a/core/src/test/scala/unit/kafka/message/MessageTest.scala b/core/src/test/scala/unit/kafka/message/MessageTest.scala deleted file mode 100755 index 2390b5b2f425b..0000000000000 --- a/core/src/test/scala/unit/kafka/message/MessageTest.scala +++ /dev/null @@ -1,143 +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.message - -import java.nio._ -import java.util.HashMap - -import org.apache.kafka.common.protocol.Errors - -import scala.collection._ -import org.junit.Assert._ -import org.scalatest.junit.JUnitSuite -import org.junit.{Before, Test} -import kafka.utils.TestUtils -import org.apache.kafka.common.utils.ByteUtils - -case class MessageTestVal(key: Array[Byte], - payload: Array[Byte], - codec: CompressionCodec, - timestamp: Long, - magicValue: Byte, - message: Message) - -class MessageTest extends JUnitSuite { - - var messages = new mutable.ArrayBuffer[MessageTestVal]() - - @Before - def setUp(): Unit = { - val keys = Array(null, "key".getBytes, "".getBytes) - val vals = Array("value".getBytes, "".getBytes, null) - val codecs = Array(NoCompressionCodec, GZIPCompressionCodec, SnappyCompressionCodec, LZ4CompressionCodec) - val timestamps = Array(Message.NoTimestamp, 0L, 1L) - val magicValues = Array(Message.MagicValue_V0, Message.MagicValue_V1) - for(k <- keys; v <- vals; codec <- codecs; t <- timestamps; mv <- magicValues) { - val timestamp = ensureValid(mv, t) - messages += MessageTestVal(k, v, codec, timestamp, mv, new Message(v, k, timestamp, codec, mv)) - } - - def ensureValid(magicValue: Byte, timestamp: Long): Long = - if (magicValue > Message.MagicValue_V0) timestamp else Message.NoTimestamp - } - - @Test - def testFieldValues(): Unit = { - for(v <- messages) { - // check payload - if(v.payload == null) { - assertTrue(v.message.isNull) - assertEquals("Payload should be null", null, v.message.payload) - } else { - TestUtils.checkEquals(ByteBuffer.wrap(v.payload), v.message.payload) - } - // check timestamp - if (v.magicValue > Message.MagicValue_V0) - assertEquals("Timestamp should be the same", v.timestamp, v.message.timestamp) - else - assertEquals("Timestamp should be the NoTimestamp", Message.NoTimestamp, v.message.timestamp) - - // check magic value - assertEquals(v.magicValue, v.message.magic) - // check key - if(v.message.hasKey) - TestUtils.checkEquals(ByteBuffer.wrap(v.key), v.message.key) - else - assertEquals(null, v.message.key) - // check compression codec - assertEquals(v.codec, v.message.compressionCodec) - } - } - - @Test - def testChecksum() { - for(v <- messages) { - assertTrue("Auto-computed checksum should be valid", v.message.isValid) - // garble checksum - val badChecksum: Int = (v.message.checksum + 1 % Int.MaxValue).toInt - ByteUtils.writeUnsignedInt(v.message.buffer, Message.CrcOffset, badChecksum) - assertFalse("Message with invalid checksum should be invalid", v.message.isValid) - } - } - - @Test - def testEquality() { - for (v <- messages) { - assertFalse("Should not equal null", v.message.equals(null)) - assertFalse("Should not equal a random string", v.message.equals("asdf")) - assertTrue("Should equal itself", v.message.equals(v.message)) - val copy = new Message(bytes = v.payload, key = v.key, v.timestamp, codec = v.codec, v.magicValue) - assertTrue("Should equal another message with the same content.", v.message.equals(copy)) - } - } - - @Test(expected = classOf[IllegalArgumentException]) - def testInvalidTimestampAndMagicValueCombination() { - new Message("hello".getBytes, 0L, Message.MagicValue_V0) - } - - @Test(expected = classOf[IllegalArgumentException]) - def testInvalidTimestamp() { - new Message("hello".getBytes, -3L, Message.MagicValue_V1) - } - - @Test(expected = classOf[IllegalArgumentException]) - def testInvalidMagicByte() { - new Message("hello".getBytes, 0L, 2.toByte) - } - - @Test - def testIsHashable() { - // this is silly, but why not - val m = new HashMap[Message, Message]() - for(v <- messages) - m.put(v.message, v.message) - for(v <- messages) - assertEquals(v.message, m.get(v.message)) - } - - @Test - def testExceptionMapping() { - val expected = Errors.CORRUPT_MESSAGE - val actual = Errors.forException(new InvalidMessageException()) - - assertEquals("InvalidMessageException should map to a corrupt message error", expected, actual) - } - -} - diff --git a/core/src/test/scala/unit/kafka/security/auth/SimpleAclAuthorizerTest.scala b/core/src/test/scala/unit/kafka/security/auth/SimpleAclAuthorizerTest.scala index 3e7f6a8020a29..1468003f3f011 100644 --- a/core/src/test/scala/unit/kafka/security/auth/SimpleAclAuthorizerTest.scala +++ b/core/src/test/scala/unit/kafka/security/auth/SimpleAclAuthorizerTest.scala @@ -17,36 +17,47 @@ package kafka.security.auth import java.net.InetAddress +import java.nio.charset.StandardCharsets.UTF_8 import java.util.UUID +import java.util.concurrent.{Executors, Semaphore, TimeUnit} +import kafka.api.{ApiVersion, KAFKA_2_0_IV0, KAFKA_2_0_IV1} import kafka.network.RequestChannel.Session import kafka.security.auth.Acl.{WildCardHost, WildCardResource} import kafka.server.KafkaConfig -import kafka.utils.TestUtils -import kafka.zk.ZooKeeperTestHarness +import kafka.utils.{CoreUtils, TestUtils} +import kafka.zk.{ZkAclStore, ZooKeeperTestHarness} +import kafka.zookeeper.{GetChildrenRequest, GetDataRequest, ZooKeeperClient} +import org.apache.kafka.common.errors.UnsupportedVersionException +import org.apache.kafka.common.resource.PatternType +import org.apache.kafka.common.resource.PatternType.{LITERAL, PREFIXED} import org.apache.kafka.common.security.auth.KafkaPrincipal +import org.apache.kafka.common.utils.Time import org.junit.Assert._ import org.junit.{After, Before, Test} class SimpleAclAuthorizerTest extends ZooKeeperTestHarness { - val allowReadAcl = Acl(Acl.WildCardPrincipal, Allow, WildCardHost, Read) - val allowWriteAcl = Acl(Acl.WildCardPrincipal, Allow, WildCardHost, Write) - val denyReadAcl = Acl(Acl.WildCardPrincipal, Deny, WildCardHost, Read) - - val wildCardResource = Resource(Topic, WildCardResource, Literal) - val prefixedResource = Resource(Topic, "foo", Prefixed) - - val simpleAclAuthorizer = new SimpleAclAuthorizer - val simpleAclAuthorizer2 = new SimpleAclAuthorizer - val testPrincipal = Acl.WildCardPrincipal - val testHostName = InetAddress.getByName("192.168.0.1") - var resource: Resource = null - val superUsers = "User:superuser1; User:superuser2" - val username = "alice" - val principal = new KafkaPrincipal(KafkaPrincipal.USER_TYPE, username) - val session = Session(principal, testHostName) - var config: KafkaConfig = null + private val allowReadAcl = Acl(Acl.WildCardPrincipal, Allow, WildCardHost, Read) + private val allowWriteAcl = Acl(Acl.WildCardPrincipal, Allow, WildCardHost, Write) + private val denyReadAcl = Acl(Acl.WildCardPrincipal, Deny, WildCardHost, Read) + + private val wildCardResource = Resource(Topic, WildCardResource, LITERAL) + private val prefixedResource = Resource(Topic, "foo", PREFIXED) + + private val simpleAclAuthorizer = new SimpleAclAuthorizer + private val simpleAclAuthorizer2 = new SimpleAclAuthorizer + private var resource: Resource = _ + private val superUsers = "User:superuser1; User:superuser2" + private val username = "alice" + private val principal = new KafkaPrincipal(KafkaPrincipal.USER_TYPE, username) + private val session = Session(principal, InetAddress.getByName("192.168.0.1")) + private var config: KafkaConfig = _ + private var zooKeeperClient: ZooKeeperClient = _ + + class CustomPrincipal(principalType: String, name: String) extends KafkaPrincipal(principalType, name) { + override def equals(o: scala.Any): Boolean = false + } @Before override def setUp() { @@ -62,19 +73,36 @@ class SimpleAclAuthorizerTest extends ZooKeeperTestHarness { config = KafkaConfig.fromProps(props) simpleAclAuthorizer.configure(config.originals) simpleAclAuthorizer2.configure(config.originals) - resource = new Resource(Topic, "foo-" + UUID.randomUUID(), Literal) + resource = Resource(Topic, "foo-" + UUID.randomUUID(), LITERAL) + + zooKeeperClient = new ZooKeeperClient(zkConnect, zkSessionTimeout, zkConnectionTimeout, zkMaxInFlightRequests, + Time.SYSTEM, "kafka.test", "SimpleAclAuthorizerTest") } @After override def tearDown(): Unit = { simpleAclAuthorizer.close() simpleAclAuthorizer2.close() + zooKeeperClient.close() super.tearDown() } @Test(expected = classOf[IllegalArgumentException]) def testAuthorizeThrowsOnNoneLiteralResource() { - simpleAclAuthorizer.authorize(session, Read, Resource(Topic, "something", Prefixed)) + simpleAclAuthorizer.authorize(session, Read, Resource(Topic, "something", PREFIXED)) + } + + @Test + def testAuthorizeWithEmptyResourceName(): Unit = { + assertFalse(simpleAclAuthorizer.authorize(session, Read, Resource(Group, "", LITERAL))) + simpleAclAuthorizer.addAcls(Set[Acl](allowReadAcl), Resource(Group, WildCardResource, LITERAL)) + assertTrue(simpleAclAuthorizer.authorize(session, Read, Resource(Group, "", LITERAL))) + } + + // Authorizing the empty resource is not supported because we create a znode with the resource name. + @Test(expected = classOf[IllegalArgumentException]) + def testEmptyAclThrowsException(): Unit = { + simpleAclAuthorizer.addAcls(Set[Acl](allowReadAcl), Resource(Group, "", LITERAL)) } @Test @@ -129,6 +157,29 @@ class SimpleAclAuthorizerTest extends ZooKeeperTestHarness { assertTrue("User3 should have WRITE access from host2", simpleAclAuthorizer.authorize(user3Session, Write, resource)) } + /** + CustomPrincipals should be compared with their principal type and name + */ + @Test + def testAllowAccessWithCustomPrincipal() { + val user = new KafkaPrincipal(KafkaPrincipal.USER_TYPE, username) + val customUserPrincipal = new CustomPrincipal(KafkaPrincipal.USER_TYPE, username) + val host1 = InetAddress.getByName("192.168.1.1") + val host2 = InetAddress.getByName("192.168.1.2") + + // user has READ access from host2 but not from host1 + val acl1 = new Acl(user, Deny, host1.getHostAddress, Read) + val acl2 = new Acl(user, Allow, host2.getHostAddress, Read) + val acls = Set[Acl](acl1, acl2) + changeAclAndVerify(Set.empty[Acl], acls, Set.empty[Acl]) + + val host1Session = Session(customUserPrincipal, host1) + val host2Session = Session(customUserPrincipal, 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)) + } + @Test def testDenyTakesPrecedence() { val user = new KafkaPrincipal(KafkaPrincipal.USER_TYPE, username) @@ -167,6 +218,19 @@ class SimpleAclAuthorizerTest extends ZooKeeperTestHarness { assertTrue("superuser always has access, no matter what acls.", simpleAclAuthorizer.authorize(session2, Read, resource)) } + /** + CustomPrincipals should be compared with their principal type and name + */ + @Test + def testSuperUserWithCustomPrincipalHasAccess(): Unit = { + val denyAllAcl = new Acl(Acl.WildCardPrincipal, Deny, WildCardHost, All) + changeAclAndVerify(Set.empty[Acl], Set[Acl](denyAllAcl), Set.empty[Acl]) + + val session = Session(new CustomPrincipal(KafkaPrincipal.USER_TYPE, "superuser1"), InetAddress.getByName("192.0.4.4")) + + assertTrue("superuser with custom principal always has access, no matter what acls.", simpleAclAuthorizer.authorize(session, Read, resource)) + } + @Test def testWildCardAcls(): Unit = { assertFalse("when acls = [], authorizer should fail close.", simpleAclAuthorizer.authorize(session, Read, resource)) @@ -234,10 +298,10 @@ class SimpleAclAuthorizerTest extends ZooKeeperTestHarness { 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, Literal) -> Set[Acl](new Acl(user2, Allow, WildCardHost, Read)), - new Resource(Cluster, Resource.WildCardResource, Literal) -> Set[Acl](new Acl(user2, Allow, host1, Read)), - new Resource(Group, Resource.WildCardResource, Literal) -> acls, - new Resource(Group, "test-ConsumerGroup", Literal) -> acls + new Resource(Topic, Resource.WildCardResource, LITERAL) -> Set[Acl](new Acl(user2, Allow, WildCardHost, Read)), + new Resource(Cluster, Resource.WildCardResource, LITERAL) -> Set[Acl](new Acl(user2, Allow, host1, Read)), + new Resource(Group, Resource.WildCardResource, LITERAL) -> acls, + new Resource(Group, "test-ConsumerGroup", LITERAL) -> acls ) resourceToAcls foreach { case (key, value) => changeAclAndVerify(Set.empty[Acl], value, Set.empty[Acl], key) } @@ -265,7 +329,7 @@ class SimpleAclAuthorizerTest extends ZooKeeperTestHarness { simpleAclAuthorizer.addAcls(acls, resource) val user2 = new KafkaPrincipal(KafkaPrincipal.USER_TYPE, "bob") - val resource1 = new Resource(Topic, "test-2", Literal) + val resource1 = Resource(Topic, "test-2", LITERAL) val acl2 = new Acl(user2, Deny, "host3", Read) val acls1 = Set[Acl](acl2) simpleAclAuthorizer.addAcls(acls1, resource1) @@ -282,9 +346,43 @@ class SimpleAclAuthorizerTest extends ZooKeeperTestHarness { } } + /** + * Verify that there is no timing window between loading ACL cache and setting + * up ZK change listener. Cache must be loaded before creating change listener + * in the authorizer to avoid the timing window. + */ + @Test + def testChangeListenerTiming() { + val configureSemaphore = new Semaphore(0) + val listenerSemaphore = new Semaphore(0) + val executor = Executors.newSingleThreadExecutor + val simpleAclAuthorizer3 = new SimpleAclAuthorizer { + override private[auth] def startZkChangeListeners(): Unit = { + configureSemaphore.release() + listenerSemaphore.acquireUninterruptibly() + super.startZkChangeListeners() + } + } + try { + val future = executor.submit(CoreUtils.runnable(simpleAclAuthorizer3.configure(config.originals))) + configureSemaphore.acquire() + val user1 = new KafkaPrincipal(KafkaPrincipal.USER_TYPE, username) + val acls = Set(new Acl(user1, Deny, "host-1", Read)) + simpleAclAuthorizer.addAcls(acls, resource) + + listenerSemaphore.release() + future.get(10, TimeUnit.SECONDS) + + assertEquals(acls, simpleAclAuthorizer3.getAcls(resource)) + } finally { + simpleAclAuthorizer3.close() + executor.shutdownNow() + } + } + @Test def testLocalConcurrentModificationOfResourceAcls() { - val commonResource = new Resource(Topic, "test", Literal) + val commonResource = Resource(Topic, "test", LITERAL) val user1 = new KafkaPrincipal(KafkaPrincipal.USER_TYPE, username) val acl1 = new Acl(user1, Allow, WildCardHost, Read) @@ -300,7 +398,7 @@ class SimpleAclAuthorizerTest extends ZooKeeperTestHarness { @Test def testDistributedConcurrentModificationOfResourceAcls() { - val commonResource = new Resource(Topic, "test", Literal) + val commonResource = Resource(Topic, "test", LITERAL) val user1 = new KafkaPrincipal(KafkaPrincipal.USER_TYPE, username) val acl1 = new Acl(user1, Allow, WildCardHost, Read) @@ -330,7 +428,7 @@ class SimpleAclAuthorizerTest extends ZooKeeperTestHarness { @Test def testHighConcurrencyModificationOfResourceAcls() { - val commonResource = new Resource(Topic, "test", Literal) + val commonResource = Resource(Topic, "test", LITERAL) val acls = (0 to 50).map { i => val useri = new KafkaPrincipal(KafkaPrincipal.USER_TYPE, i.toString) @@ -513,43 +611,138 @@ class SimpleAclAuthorizerTest extends ZooKeeperTestHarness { @Test def testAuthorizeWithPrefixedResource(): Unit = { - simpleAclAuthorizer.addAcls(Set[Acl](denyReadAcl), Resource(Topic, "a_other", Literal)) - simpleAclAuthorizer.addAcls(Set[Acl](denyReadAcl), Resource(Topic, "a_other", Prefixed)) - simpleAclAuthorizer.addAcls(Set[Acl](denyReadAcl), Resource(Topic, "foo-" + UUID.randomUUID(), Prefixed)) - simpleAclAuthorizer.addAcls(Set[Acl](denyReadAcl), Resource(Topic, "foo-" + UUID.randomUUID(), Prefixed)) - simpleAclAuthorizer.addAcls(Set[Acl](denyReadAcl), Resource(Topic, "foo-" + UUID.randomUUID() + "-zzz", Prefixed)) - simpleAclAuthorizer.addAcls(Set[Acl](denyReadAcl), Resource(Topic, "fooo-" + UUID.randomUUID(), Prefixed)) - simpleAclAuthorizer.addAcls(Set[Acl](denyReadAcl), Resource(Topic, "fo-" + UUID.randomUUID(), Prefixed)) - simpleAclAuthorizer.addAcls(Set[Acl](denyReadAcl), Resource(Topic, "fop-" + UUID.randomUUID(), Prefixed)) - simpleAclAuthorizer.addAcls(Set[Acl](denyReadAcl), Resource(Topic, "fon-" + UUID.randomUUID(), Prefixed)) - simpleAclAuthorizer.addAcls(Set[Acl](denyReadAcl), Resource(Topic, "fon-", Prefixed)) - simpleAclAuthorizer.addAcls(Set[Acl](denyReadAcl), Resource(Topic, "z_other", Prefixed)) - simpleAclAuthorizer.addAcls(Set[Acl](denyReadAcl), Resource(Topic, "z_other", Literal)) + simpleAclAuthorizer.addAcls(Set[Acl](denyReadAcl), Resource(Topic, "a_other", LITERAL)) + simpleAclAuthorizer.addAcls(Set[Acl](denyReadAcl), Resource(Topic, "a_other", PREFIXED)) + simpleAclAuthorizer.addAcls(Set[Acl](denyReadAcl), Resource(Topic, "foo-" + UUID.randomUUID(), PREFIXED)) + simpleAclAuthorizer.addAcls(Set[Acl](denyReadAcl), Resource(Topic, "foo-" + UUID.randomUUID(), PREFIXED)) + simpleAclAuthorizer.addAcls(Set[Acl](denyReadAcl), Resource(Topic, "foo-" + UUID.randomUUID() + "-zzz", PREFIXED)) + simpleAclAuthorizer.addAcls(Set[Acl](denyReadAcl), Resource(Topic, "fooo-" + UUID.randomUUID(), PREFIXED)) + simpleAclAuthorizer.addAcls(Set[Acl](denyReadAcl), Resource(Topic, "fo-" + UUID.randomUUID(), PREFIXED)) + simpleAclAuthorizer.addAcls(Set[Acl](denyReadAcl), Resource(Topic, "fop-" + UUID.randomUUID(), PREFIXED)) + simpleAclAuthorizer.addAcls(Set[Acl](denyReadAcl), Resource(Topic, "fon-" + UUID.randomUUID(), PREFIXED)) + simpleAclAuthorizer.addAcls(Set[Acl](denyReadAcl), Resource(Topic, "fon-", PREFIXED)) + simpleAclAuthorizer.addAcls(Set[Acl](denyReadAcl), Resource(Topic, "z_other", PREFIXED)) + simpleAclAuthorizer.addAcls(Set[Acl](denyReadAcl), Resource(Topic, "z_other", LITERAL)) simpleAclAuthorizer.addAcls(Set[Acl](allowReadAcl), prefixedResource) assertTrue(simpleAclAuthorizer.authorize(session, Read, resource)) } + @Test + def testSingleCharacterResourceAcls(): Unit = { + simpleAclAuthorizer.addAcls(Set[Acl](allowReadAcl), Resource(Topic, "f", LITERAL)) + assertTrue(simpleAclAuthorizer.authorize(session, Read, Resource(Topic, "f", LITERAL))) + assertFalse(simpleAclAuthorizer.authorize(session, Read, Resource(Topic, "foo", LITERAL))) + + simpleAclAuthorizer.addAcls(Set[Acl](allowReadAcl), Resource(Topic, "_", PREFIXED)) + assertTrue(simpleAclAuthorizer.authorize(session, Read, Resource(Topic, "_foo", LITERAL))) + assertTrue(simpleAclAuthorizer.authorize(session, Read, Resource(Topic, "_", LITERAL))) + assertFalse(simpleAclAuthorizer.authorize(session, Read, Resource(Topic, "foo_", LITERAL))) + } + @Test def testGetAclsPrincipal(): Unit = { - assertEquals(0, simpleAclAuthorizer.getAcls(principal).size) + val aclOnSpecificPrincipal = new Acl(principal, Allow, WildCardHost, Write) + simpleAclAuthorizer.addAcls(Set[Acl](aclOnSpecificPrincipal), resource) + + assertEquals("acl on specific should not be returned for wildcard request", + 0, simpleAclAuthorizer.getAcls(Acl.WildCardPrincipal).size) + assertEquals("acl on specific should be returned for specific request", + 1, simpleAclAuthorizer.getAcls(principal).size) + assertEquals("acl on specific should be returned for different principal instance", + 1, simpleAclAuthorizer.getAcls(new KafkaPrincipal(principal.getPrincipalType, principal.getName)).size) + + simpleAclAuthorizer.removeAcls(resource) + val aclOnWildcardPrincipal = new Acl(Acl.WildCardPrincipal, Allow, WildCardHost, Write) + simpleAclAuthorizer.addAcls(Set[Acl](aclOnWildcardPrincipal), resource) + + assertEquals("acl on wildcard should be returned for wildcard request", + 1, simpleAclAuthorizer.getAcls(Acl.WildCardPrincipal).size) + assertEquals("acl on wildcard should not be returned for specific request", + 0, simpleAclAuthorizer.getAcls(principal).size) + } + + @Test(expected = classOf[UnsupportedVersionException]) + def testThrowsOnAddPrefixedAclIfInterBrokerProtocolVersionTooLow(): Unit = { + givenAuthorizerWithProtocolVersion(Option(KAFKA_2_0_IV0)) + simpleAclAuthorizer.addAcls(Set[Acl](denyReadAcl), Resource(Topic, "z_other", PREFIXED)) + } + + @Test + def testWritesExtendedAclChangeEventIfInterBrokerProtocolNotSet(): Unit = { + givenAuthorizerWithProtocolVersion(Option.empty) + val resource = Resource(Topic, "z_other", PREFIXED) + val expected = new String(ZkAclStore(PREFIXED).changeStore.createChangeNode(resource).bytes, UTF_8) + + simpleAclAuthorizer.addAcls(Set[Acl](denyReadAcl), resource) + + val actual = getAclChangeEventAsString(PREFIXED) + + assertEquals(expected, actual) + } + + @Test + def testWritesExtendedAclChangeEventWhenInterBrokerProtocolAtLeastKafkaV2(): Unit = { + givenAuthorizerWithProtocolVersion(Option(KAFKA_2_0_IV1)) + val resource = Resource(Topic, "z_other", PREFIXED) + val expected = new String(ZkAclStore(PREFIXED).changeStore.createChangeNode(resource).bytes, UTF_8) + + simpleAclAuthorizer.addAcls(Set[Acl](denyReadAcl), resource) + + val actual = getAclChangeEventAsString(PREFIXED) + + assertEquals(expected, actual) + } - val acl1 = new Acl(principal, Allow, WildCardHost, Write) - simpleAclAuthorizer.addAcls(Set[Acl](acl1), resource) - assertEquals(1, simpleAclAuthorizer.getAcls(principal).size) + @Test + def testWritesLiteralWritesLiteralAclChangeEventWhenInterBrokerProtocolLessThanKafkaV2eralAclChangesForOlderProtocolVersions(): Unit = { + givenAuthorizerWithProtocolVersion(Option(KAFKA_2_0_IV0)) + val resource = Resource(Topic, "z_other", LITERAL) + val expected = new String(ZkAclStore(LITERAL).changeStore.createChangeNode(resource).bytes, UTF_8) + + simpleAclAuthorizer.addAcls(Set[Acl](denyReadAcl), resource) + + val actual = getAclChangeEventAsString(LITERAL) + + assertEquals(expected, actual) + } + + @Test + def testWritesLiteralAclChangeEventWhenInterBrokerProtocolIsKafkaV2(): Unit = { + givenAuthorizerWithProtocolVersion(Option(KAFKA_2_0_IV1)) + val resource = Resource(Topic, "z_other", LITERAL) + val expected = new String(ZkAclStore(LITERAL).changeStore.createChangeNode(resource).bytes, UTF_8) + + simpleAclAuthorizer.addAcls(Set[Acl](denyReadAcl), resource) + + val actual = getAclChangeEventAsString(LITERAL) + + assertEquals(expected, actual) + } + + private def givenAuthorizerWithProtocolVersion(protocolVersion: Option[ApiVersion]) { + simpleAclAuthorizer.close() + + val props = TestUtils.createBrokerConfig(0, zkConnect) + props.put(SimpleAclAuthorizer.SuperUsersProp, superUsers) + protocolVersion.foreach(version => props.put(KafkaConfig.InterBrokerProtocolVersionProp, version.toString)) + + config = KafkaConfig.fromProps(props) + + simpleAclAuthorizer.configure(config.originals) + } - simpleAclAuthorizer.addAcls(Set[Acl](acl1), new Resource(Topic, Acl.WildCardResource, Literal)) - assertEquals(2, simpleAclAuthorizer.getAcls(principal).size) + private def getAclChangeEventAsString(patternType: PatternType) = { + val store = ZkAclStore(patternType) + val children = zooKeeperClient.handleRequest(GetChildrenRequest(store.changeStore.aclChangePath)) + children.maybeThrow() + assertEquals("Expecting 1 change event", 1, children.children.size) - val acl2 = new Acl(Acl.WildCardPrincipal, Allow, WildCardHost, Write) - simpleAclAuthorizer.addAcls(Set[Acl](acl1), new Resource(Group, "groupA", Literal)) - assertEquals(3, simpleAclAuthorizer.getAcls(principal).size) + val data = zooKeeperClient.handleRequest(GetDataRequest(s"${store.changeStore.aclChangePath}/${children.children.head}")) + data.maybeThrow() - // add prefixed principal acl on wildcard group name - val acl3 = new Acl(new KafkaPrincipal(KafkaPrincipal.USER_TYPE, principal.getName.charAt(0) + WildCardResource), Allow, WildCardHost, Write) - simpleAclAuthorizer.addAcls(Set[Acl](acl1), new Resource(Group, Acl.WildCardResource, Literal)) - assertEquals(4, simpleAclAuthorizer.getAcls(principal).size) + new String(data.data, UTF_8) } private def changeAclAndVerify(originalAcls: Set[Acl], addedAcls: Set[Acl], removedAcls: Set[Acl], resource: Resource = resource): Set[Acl] = { diff --git a/core/src/test/scala/unit/kafka/security/auth/ZkAuthorizationTest.scala b/core/src/test/scala/unit/kafka/security/auth/ZkAuthorizationTest.scala index 19fa19dafbc27..1cdbe4b2a0ea9 100644 --- a/core/src/test/scala/unit/kafka/security/auth/ZkAuthorizationTest.scala +++ b/core/src/test/scala/unit/kafka/security/auth/ZkAuthorizationTest.scala @@ -19,10 +19,10 @@ package kafka.security.auth import kafka.admin.ZkSecurityMigrator import kafka.utils.{CoreUtils, Logging, TestUtils, ZkUtils} -import kafka.zk.ZooKeeperTestHarness +import kafka.zk.{ConsumerPathZNode, ZooKeeperTestHarness} import org.apache.kafka.common.KafkaException import org.apache.kafka.common.security.JaasUtils -import org.apache.zookeeper.data.ACL +import org.apache.zookeeper.data.{ACL, Stat} import org.junit.Assert._ import org.junit.{After, Before, Test} @@ -304,4 +304,12 @@ class ZkAuthorizationTest extends ZooKeeperTestHarness with Logging { } } } + + @Test + def testConsumerOffsetPathAcls(): Unit = { + zkClient.makeSurePersistentPathExists(ConsumerPathZNode.path) + + val consumerPathAcls = zkClient.currentZooKeeper.getACL(ConsumerPathZNode.path, new Stat()) + assertTrue("old consumer znode path acls are not open", consumerPathAcls.asScala.forall(TestUtils.isAclUnsecure)) + } } diff --git a/core/src/test/scala/unit/kafka/security/token/delegation/DelegationTokenManagerTest.scala b/core/src/test/scala/unit/kafka/security/token/delegation/DelegationTokenManagerTest.scala index eec7175983ef3..b8d4376c54abd 100644 --- a/core/src/test/scala/unit/kafka/security/token/delegation/DelegationTokenManagerTest.scala +++ b/core/src/test/scala/unit/kafka/security/token/delegation/DelegationTokenManagerTest.scala @@ -26,23 +26,26 @@ import kafka.security.auth.Acl.WildCardHost import kafka.security.auth._ import kafka.server.{CreateTokenResult, Defaults, DelegationTokenManager, KafkaConfig} import kafka.utils.TestUtils -import kafka.zk.ZooKeeperTestHarness +import kafka.zk.{KafkaZkClient, ZooKeeperTestHarness} import org.apache.kafka.common.protocol.Errors +import org.apache.kafka.common.resource.PatternType.LITERAL import org.apache.kafka.common.security.auth.KafkaPrincipal import org.apache.kafka.common.security.scram.internals.ScramMechanism import org.apache.kafka.common.security.token.delegation.internals.DelegationTokenCache import org.apache.kafka.common.security.token.delegation.{DelegationToken, TokenInformation} -import org.apache.kafka.common.utils.{MockTime, SecurityUtils} +import org.apache.kafka.common.utils.{MockTime, SecurityUtils, Time} import org.junit.Assert._ import org.junit.{After, Before, Test} import scala.collection.JavaConverters._ +import scala.collection.mutable.Buffer class DelegationTokenManagerTest extends ZooKeeperTestHarness { val time = new MockTime() val owner = SecurityUtils.parseKafkaPrincipal("User:owner") val renewer = List(SecurityUtils.parseKafkaPrincipal("User:renewer1")) + val tokenManagers = Buffer[DelegationTokenManager]() val masterKey = "masterKey" val maxLifeTimeMsDefault = Defaults.DelegationTokenMaxLifeTimeMsDefault @@ -63,11 +66,17 @@ class DelegationTokenManagerTest extends ZooKeeperTestHarness { tokenCache = new DelegationTokenCache(ScramMechanism.mechanismNames()) } + @After + override def tearDown(): Unit = { + tokenManagers.foreach(_.shutdown()) + super.tearDown() + } + @Test def testTokenRequestsWithDelegationTokenDisabled(): Unit = { val props: Properties = TestUtils.createBrokerConfig(0, zkConnect) val config = KafkaConfig.fromProps(props) - val tokenManager = new DelegationTokenManager(config, tokenCache, time, zkClient) + val tokenManager = createDelegationTokenManager(config, tokenCache, time, zkClient) tokenManager.createToken(owner, renewer, -1, createTokenResultCallBack) assertEquals(Errors.DELEGATION_TOKEN_AUTH_DISABLED, createTokenResult.error) @@ -83,7 +92,7 @@ class DelegationTokenManagerTest extends ZooKeeperTestHarness { @Test def testCreateToken(): Unit = { val config = KafkaConfig.fromProps(props) - val tokenManager = new DelegationTokenManager(config, tokenCache, time, zkClient) + val tokenManager = createDelegationTokenManager(config, tokenCache, time, zkClient) tokenManager.startup tokenManager.createToken(owner, renewer, -1 , createTokenResultCallBack) @@ -100,7 +109,7 @@ class DelegationTokenManagerTest extends ZooKeeperTestHarness { @Test def testRenewToken(): Unit = { val config = KafkaConfig.fromProps(props) - val tokenManager = new DelegationTokenManager(config, tokenCache, time, zkClient) + val tokenManager = createDelegationTokenManager(config, tokenCache, time, zkClient) tokenManager.startup tokenManager.createToken(owner, renewer, -1 , createTokenResultCallBack) @@ -148,7 +157,7 @@ class DelegationTokenManagerTest extends ZooKeeperTestHarness { @Test def testExpireToken(): Unit = { val config = KafkaConfig.fromProps(props) - val tokenManager = new DelegationTokenManager(config, tokenCache, time, zkClient) + val tokenManager = createDelegationTokenManager(config, tokenCache, time, zkClient) tokenManager.startup tokenManager.createToken(owner, renewer, -1 , createTokenResultCallBack) @@ -200,7 +209,7 @@ class DelegationTokenManagerTest extends ZooKeeperTestHarness { var hostSession = new Session(owner1, InetAddress.getByName("192.168.1.1")) - val tokenManager = new DelegationTokenManager(config, tokenCache, time, zkClient) + val tokenManager = createDelegationTokenManager(config, tokenCache, time, zkClient) tokenManager.startup //create tokens @@ -242,7 +251,7 @@ class DelegationTokenManagerTest extends ZooKeeperTestHarness { //get all tokens for multiple owners (owner1, renewer4) and with permission var acl = new Acl(owner1, Allow, WildCardHost, Describe) - simpleAclAuthorizer.addAcls(Set(acl), new Resource(kafka.security.auth.DelegationToken, tokenId3, Literal)) + simpleAclAuthorizer.addAcls(Set(acl), Resource(kafka.security.auth.DelegationToken, tokenId3, LITERAL)) tokens = getTokens(tokenManager, simpleAclAuthorizer, hostSession, owner1, List(owner1, renewer4)) assert(tokens.size == 3) @@ -257,7 +266,7 @@ class DelegationTokenManagerTest extends ZooKeeperTestHarness { //get all tokens for multiple owners (renewer2, renewer3) which are token renewers principals and with permissions hostSession = new Session(renewer2, InetAddress.getByName("192.168.1.1")) acl = new Acl(renewer2, Allow, WildCardHost, Describe) - simpleAclAuthorizer.addAcls(Set(acl), new Resource(kafka.security.auth.DelegationToken, tokenId2, Literal)) + simpleAclAuthorizer.addAcls(Set(acl), Resource(kafka.security.auth.DelegationToken, tokenId2, LITERAL)) tokens = getTokens(tokenManager, simpleAclAuthorizer, hostSession, renewer2, List(renewer2, renewer3)) assert(tokens.size == 2) @@ -271,7 +280,7 @@ class DelegationTokenManagerTest extends ZooKeeperTestHarness { List() } else { - def authorizeToken(tokenId: String) = simpleAclAuthorizer.authorize(hostSession, Describe, new Resource(kafka.security.auth.DelegationToken, tokenId, Literal)) + def authorizeToken(tokenId: String) = simpleAclAuthorizer.authorize(hostSession, Describe, Resource(kafka.security.auth.DelegationToken, tokenId, LITERAL)) def eligible(token: TokenInformation) = DelegationTokenManager.filterToken(requestPrincipal, Option(requestedOwners), token, authorizeToken) tokenManager.getTokens(eligible) } @@ -280,7 +289,7 @@ class DelegationTokenManagerTest extends ZooKeeperTestHarness { @Test def testPeriodicTokenExpiry(): Unit = { val config = KafkaConfig.fromProps(props) - val tokenManager = new DelegationTokenManager(config, tokenCache, time, zkClient) + val tokenManager = createDelegationTokenManager(config, tokenCache, time, zkClient) tokenManager.startup //create tokens @@ -296,11 +305,6 @@ class DelegationTokenManagerTest extends ZooKeeperTestHarness { } - @After - override def tearDown(): Unit = { - super.tearDown() - } - private def createTokenResultCallBack(ret: CreateTokenResult): Unit = { createTokenResult = ret } @@ -309,4 +313,11 @@ class DelegationTokenManagerTest extends ZooKeeperTestHarness { error = ret expiryTimeStamp = timeStamp } + + private def createDelegationTokenManager(config: KafkaConfig, tokenCache: DelegationTokenCache, + time: Time, zkClient: KafkaZkClient): DelegationTokenManager = { + val tokenManager = new DelegationTokenManager(config, tokenCache, time, zkClient) + tokenManagers += tokenManager + tokenManager + } } diff --git a/core/src/test/scala/unit/kafka/server/AbstractFetcherThreadTest.scala b/core/src/test/scala/unit/kafka/server/AbstractFetcherThreadTest.scala index 7ad937194f577..3ee5815a1a95f 100644 --- a/core/src/test/scala/unit/kafka/server/AbstractFetcherThreadTest.scala +++ b/core/src/test/scala/unit/kafka/server/AbstractFetcherThreadTest.scala @@ -125,8 +125,8 @@ class AbstractFetcherThreadTest { override protected def buildFetchRequest(partitionMap: collection.Seq[(TopicPartition, PartitionFetchState)]): ResultWithPartitions[DummyFetchRequest] = ResultWithPartitions(new DummyFetchRequest(partitionMap.map { case (k, v) => (k, v.fetchOffset) }.toMap), Set()) - override def buildLeaderEpochRequest(allPartitions: Seq[(TopicPartition, PartitionFetchState)]): ResultWithPartitions[Map[TopicPartition, Int]] = { - ResultWithPartitions(Map(), Set()) + override def buildLeaderEpochRequest(allPartitions: Seq[(TopicPartition, PartitionFetchState)]): (Map[TopicPartition, Int], Set[TopicPartition]) = { + (Map(), Set()) } override def fetchEpochsFromLeader(partitions: Map[TopicPartition, Int]): Map[TopicPartition, EpochEndOffset] = { Map() } diff --git a/core/src/test/scala/unit/kafka/server/FetchRequestDownConversionConfigTest.scala b/core/src/test/scala/unit/kafka/server/FetchRequestDownConversionConfigTest.scala new file mode 100644 index 0000000000000..0a448813aebb0 --- /dev/null +++ b/core/src/test/scala/unit/kafka/server/FetchRequestDownConversionConfigTest.scala @@ -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 unit.kafka.server + +import java.util +import java.util.Properties + +import kafka.log.LogConfig +import kafka.server.{BaseRequestTest, KafkaConfig} +import kafka.utils.TestUtils +import org.apache.kafka.clients.producer.{KafkaProducer, ProducerRecord} +import org.apache.kafka.common.TopicPartition +import org.apache.kafka.common.protocol.{ApiKeys, Errors} +import org.apache.kafka.common.record.MemoryRecords +import org.apache.kafka.common.requests.{FetchRequest, FetchResponse} +import org.apache.kafka.common.serialization.StringSerializer +import org.junit.Assert._ +import org.junit.Test + +class FetchRequestDownConversionConfigTest extends BaseRequestTest { + private var producer: KafkaProducer[String, String] = null + override def numBrokers: Int = 1 + + override def setUp(): Unit = { + super.setUp() + initProducer() + } + + override def tearDown(): Unit = { + if (producer != null) + producer.close() + super.tearDown() + } + + override protected def propertyOverrides(properties: Properties): Unit = { + super.propertyOverrides(properties) + properties.put(KafkaConfig.LogMessageDownConversionEnableProp, "false") + } + + private def initProducer(): Unit = { + producer = TestUtils.createProducer(TestUtils.getBrokerListStrFromServers(servers), + retries = 5, keySerializer = new StringSerializer, valueSerializer = new StringSerializer) + } + + private def createTopics(numTopics: Int, numPartitions: Int, + configs: Map[String, String] = Map.empty, topicSuffixStart: Int = 0): Map[TopicPartition, Int] = { + val topics = (0 until numTopics).map(t => s"topic${t + topicSuffixStart}") + val topicConfig = new Properties + topicConfig.setProperty(LogConfig.MinInSyncReplicasProp, 1.toString) + configs.foreach { case (k, v) => topicConfig.setProperty(k, v) } + topics.flatMap { topic => + val partitionToLeader = createTopic(topic, numPartitions = numPartitions, replicationFactor = 1, + topicConfig = topicConfig) + partitionToLeader.map { case (partition, leader) => new TopicPartition(topic, partition) -> leader } + }.toMap + } + + private def createPartitionMap(maxPartitionBytes: Int, topicPartitions: Seq[TopicPartition], + offsetMap: Map[TopicPartition, Long] = Map.empty): util.LinkedHashMap[TopicPartition, FetchRequest.PartitionData] = { + val partitionMap = new util.LinkedHashMap[TopicPartition, FetchRequest.PartitionData] + topicPartitions.foreach { tp => + partitionMap.put(tp, new FetchRequest.PartitionData(offsetMap.getOrElse(tp, 0), 0L, maxPartitionBytes)) + } + partitionMap + } + + private def sendFetchRequest(leaderId: Int, request: FetchRequest): FetchResponse[MemoryRecords] = { + val response = connectAndSend(request, ApiKeys.FETCH, destination = brokerSocketServer(leaderId)) + FetchResponse.parse(response, request.version) + } + + /** + * Tests that fetch request that require down-conversion returns with an error response when down-conversion is disabled on broker. + */ + @Test + def testV1FetchWithDownConversionDisabled(): Unit = { + val topicMap = createTopics(numTopics = 5, numPartitions = 1) + val topicPartitions = topicMap.keySet.toSeq + topicPartitions.foreach(tp => producer.send(new ProducerRecord(tp.topic(), "key", "value")).get()) + val fetchRequest = FetchRequest.Builder.forConsumer(Int.MaxValue, 0, createPartitionMap(1024, + topicPartitions)).build(1) + val fetchResponse = sendFetchRequest(topicMap.head._2, fetchRequest) + topicPartitions.foreach(tp => assertEquals(Errors.UNSUPPORTED_VERSION, fetchResponse.responseData().get(tp).error)) + } + + /** + * Tests that "message.downconversion.enable" has no effect when down-conversion is not required. + */ + @Test + def testLatestFetchWithDownConversionDisabled(): Unit = { + val topicMap = createTopics(numTopics = 5, numPartitions = 1) + val topicPartitions = topicMap.keySet.toSeq + topicPartitions.foreach(tp => producer.send(new ProducerRecord(tp.topic(), "key", "value")).get()) + val fetchRequest = FetchRequest.Builder.forConsumer(Int.MaxValue, 0, createPartitionMap(1024, + topicPartitions)).build() + val fetchResponse = sendFetchRequest(topicMap.head._2, fetchRequest) + topicPartitions.foreach(tp => assertEquals(Errors.NONE, fetchResponse.responseData().get(tp).error)) + } + + /** + * Tests that "message.downconversion.enable" can be set at topic level, and its configuration is obeyed for client + * fetch requests. + */ + @Test + def testV1FetchWithTopicLevelOverrides(): Unit = { + // create topics with default down-conversion configuration (i.e. conversion disabled) + val conversionDisabledTopicsMap = createTopics(numTopics = 5, numPartitions = 1, topicSuffixStart = 0) + val conversionDisabledTopicPartitions = conversionDisabledTopicsMap.keySet.toSeq + + // create topics with down-conversion configuration enabled + val topicConfig = Map(LogConfig.MessageDownConversionEnableProp -> "true") + val conversionEnabledTopicsMap = createTopics(numTopics = 5, numPartitions = 1, topicConfig, topicSuffixStart = 5) + val conversionEnabledTopicPartitions = conversionEnabledTopicsMap.keySet.toSeq + + val allTopics = conversionDisabledTopicPartitions ++ conversionEnabledTopicPartitions + val leaderId = conversionDisabledTopicsMap.head._2 + + allTopics.foreach(tp => producer.send(new ProducerRecord(tp.topic(), "key", "value")).get()) + val fetchRequest = FetchRequest.Builder.forConsumer(Int.MaxValue, 0, createPartitionMap(1024, + allTopics)).build(1) + val fetchResponse = sendFetchRequest(leaderId, fetchRequest) + + conversionDisabledTopicPartitions.foreach(tp => assertEquals(Errors.UNSUPPORTED_VERSION, fetchResponse.responseData().get(tp).error)) + conversionEnabledTopicPartitions.foreach(tp => assertEquals(Errors.NONE, fetchResponse.responseData().get(tp).error)) + } + + /** + * Tests that "message.downconversion.enable" has no effect on fetch requests from replicas. + */ + @Test + def testV1FetchFromReplica(): Unit = { + // create topics with default down-conversion configuration (i.e. conversion disabled) + val conversionDisabledTopicsMap = createTopics(numTopics = 5, numPartitions = 1, topicSuffixStart = 0) + val conversionDisabledTopicPartitions = conversionDisabledTopicsMap.keySet.toSeq + + // create topics with down-conversion configuration enabled + val topicConfig = Map(LogConfig.MessageDownConversionEnableProp -> "true") + val conversionEnabledTopicsMap = createTopics(numTopics = 5, numPartitions = 1, topicConfig, topicSuffixStart = 5) + val conversionEnabledTopicPartitions = conversionEnabledTopicsMap.keySet.toSeq + + val allTopicPartitions = conversionDisabledTopicPartitions ++ conversionEnabledTopicPartitions + val leaderId = conversionDisabledTopicsMap.head._2 + + allTopicPartitions.foreach(tp => producer.send(new ProducerRecord(tp.topic(), "key", "value")).get()) + val fetchRequest = FetchRequest.Builder.forReplica(1, 1, Int.MaxValue, 0, + createPartitionMap(1024, allTopicPartitions)).build() + val fetchResponse = sendFetchRequest(leaderId, fetchRequest) + + allTopicPartitions.foreach(tp => assertEquals(Errors.NONE, fetchResponse.responseData().get(tp).error)) + } +} diff --git a/core/src/test/scala/unit/kafka/server/FetchRequestTest.scala b/core/src/test/scala/unit/kafka/server/FetchRequestTest.scala index 424b8c79fe424..b995025cc5021 100644 --- a/core/src/test/scala/unit/kafka/server/FetchRequestTest.scala +++ b/core/src/test/scala/unit/kafka/server/FetchRequestTest.scala @@ -26,8 +26,8 @@ import kafka.utils.TestUtils import org.apache.kafka.clients.producer.{KafkaProducer, ProducerConfig, ProducerRecord} import org.apache.kafka.common.TopicPartition import org.apache.kafka.common.protocol.{ApiKeys, Errors} -import org.apache.kafka.common.record.{MemoryRecords, Record, RecordBatch, Records} -import org.apache.kafka.common.requests.{FetchRequest, FetchResponse, FetchMetadata => JFetchMetadata} +import org.apache.kafka.common.record.{MemoryRecords, Record, RecordBatch} +import org.apache.kafka.common.requests.{FetchRequest, FetchResponse, IsolationLevel, FetchMetadata => JFetchMetadata} import org.apache.kafka.common.serialization.{ByteArraySerializer, StringSerializer} import org.junit.Assert._ import org.junit.Test @@ -69,7 +69,7 @@ class FetchRequestTest extends BaseRequestTest { } private def initProducer(): Unit = { - producer = TestUtils.createNewProducer(TestUtils.getBrokerListStrFromServers(servers), + producer = TestUtils.createProducer(TestUtils.getBrokerListStrFromServers(servers), retries = 5, keySerializer = new StringSerializer, valueSerializer = new StringSerializer) } @@ -169,6 +169,22 @@ class FetchRequestTest extends BaseRequestTest { assertEquals(0, records(partitionData).map(_.sizeInBytes).sum) } + @Test + def testFetchRequestV4WithReadCommitted(): Unit = { + initProducer() + val maxPartitionBytes = 200 + val (topicPartition, leaderId) = createTopics(numTopics = 1, numPartitions = 1).head + producer.send(new ProducerRecord(topicPartition.topic, topicPartition.partition, + "key", new String(new Array[Byte](maxPartitionBytes + 1)))).get + val fetchRequest = FetchRequest.Builder.forConsumer(Int.MaxValue, 0, createPartitionMap(maxPartitionBytes, + Seq(topicPartition))).isolationLevel(IsolationLevel.READ_COMMITTED).build(4) + val fetchResponse = sendFetchRequest(leaderId, fetchRequest) + val partitionData = fetchResponse.responseData.get(topicPartition) + assertEquals(Errors.NONE, partitionData.error) + assertTrue(partitionData.lastStableOffset > 0) + assertTrue(records(partitionData).map(_.sizeInBytes).sum > 0) + } + @Test def testFetchRequestToNonReplica(): Unit = { val topic = "topic" @@ -203,7 +219,7 @@ class FetchRequestTest extends BaseRequestTest { val batchSize = 4 * msgValueLen val propsOverride = new Properties propsOverride.put(ProducerConfig.BATCH_SIZE_CONFIG, batchSize.toString) - val producer = TestUtils.createNewProducer(TestUtils.getBrokerListStrFromServers(servers), + val producer = TestUtils.createProducer(TestUtils.getBrokerListStrFromServers(servers), retries = 5, lingerMs = Long.MaxValue, keySerializer = new StringSerializer, valueSerializer = new ByteArraySerializer, props = Some(propsOverride)) val bytes = new Array[Byte](msgValueLen) @@ -261,7 +277,7 @@ class FetchRequestTest extends BaseRequestTest { @Test def testDownConversionFromBatchedToUnbatchedRespectsOffset(): Unit = { // Increase linger so that we have control over the batches created - producer = TestUtils.createNewProducer(TestUtils.getBrokerListStrFromServers(servers), + producer = TestUtils.createProducer(TestUtils.getBrokerListStrFromServers(servers), retries = 5, keySerializer = new StringSerializer, valueSerializer = new StringSerializer, lingerMs = 300 * 1000) @@ -426,7 +442,7 @@ class FetchRequestTest extends BaseRequestTest { } private def createTopics(numTopics: Int, numPartitions: Int, configs: Map[String, String] = Map.empty): Map[TopicPartition, Int] = { - val topics = (0 until numPartitions).map(t => s"topic$t") + val topics = (0 until numTopics).map(t => s"topic$t") val topicConfig = new Properties topicConfig.setProperty(LogConfig.MinInSyncReplicasProp, 2.toString) configs.foreach { case (k, v) => topicConfig.setProperty(k, v) } diff --git a/core/src/test/scala/unit/kafka/server/FetchSessionTest.scala b/core/src/test/scala/unit/kafka/server/FetchSessionTest.scala index 84efa6b684d54..c4a96254eda28 100755 --- a/core/src/test/scala/unit/kafka/server/FetchSessionTest.scala +++ b/core/src/test/scala/unit/kafka/server/FetchSessionTest.scala @@ -22,7 +22,7 @@ import java.util.Collections import kafka.utils.MockTime import org.apache.kafka.common.TopicPartition import org.apache.kafka.common.protocol.Errors -import org.apache.kafka.common.record.{AbstractRecords, Records} +import org.apache.kafka.common.record.Records import org.apache.kafka.common.requests.FetchMetadata.{FINAL_EPOCH, INVALID_SESSION_ID} import org.apache.kafka.common.requests.{FetchRequest, FetchResponse, FetchMetadata => JFetchMetadata} import org.junit.Assert._ @@ -49,7 +49,7 @@ class FetchSessionTest { assertTrue("Missing session " + i + " out of " + sessionIds.size + "(" + sessionId + ")", cache.get(sessionId).isDefined) } - assertEquals(sessionIds.size, cache.size()) + assertEquals(sessionIds.size, cache.size) } private def dummyCreate(size: Int)() = { @@ -63,7 +63,7 @@ class FetchSessionTest { @Test def testSessionCache(): Unit = { val cache = new FetchSessionCache(3, 100) - assertEquals(0, cache.size()) + assertEquals(0, cache.size) val id1 = cache.maybeCreateSession(0, false, 10, dummyCreate(10)) val id2 = cache.maybeCreateSession(10, false, 20, dummyCreate(20)) val id3 = cache.maybeCreateSession(20, false, 30, dummyCreate(30)) @@ -86,44 +86,44 @@ class FetchSessionTest { @Test def testResizeCachedSessions(): Unit = { val cache = new FetchSessionCache(2, 100) - assertEquals(0, cache.totalPartitions()) - assertEquals(0, cache.size()) - assertEquals(0, cache.evictionsMeter.count()) + assertEquals(0, cache.totalPartitions) + assertEquals(0, cache.size) + assertEquals(0, cache.evictionsMeter.count) val id1 = cache.maybeCreateSession(0, false, 2, dummyCreate(2)) assertTrue(id1 > 0) assertCacheContains(cache, id1) val session1 = cache.get(id1).get - assertEquals(2, session1.size()) - assertEquals(2, cache.totalPartitions()) - assertEquals(1, cache.size()) - assertEquals(0, cache.evictionsMeter.count()) + assertEquals(2, session1.size) + assertEquals(2, cache.totalPartitions) + assertEquals(1, cache.size) + assertEquals(0, cache.evictionsMeter.count) val id2 = cache.maybeCreateSession(0, false, 4, dummyCreate(4)) val session2 = cache.get(id2).get assertTrue(id2 > 0) assertCacheContains(cache, id1, id2) - assertEquals(6, cache.totalPartitions()) - assertEquals(2, cache.size()) - assertEquals(0, cache.evictionsMeter.count()) + assertEquals(6, cache.totalPartitions) + assertEquals(2, cache.size) + assertEquals(0, cache.evictionsMeter.count) cache.touch(session1, 200) cache.touch(session2, 200) val id3 = cache.maybeCreateSession(200, false, 5, dummyCreate(5)) assertTrue(id3 > 0) assertCacheContains(cache, id2, id3) - assertEquals(9, cache.totalPartitions()) - assertEquals(2, cache.size()) - assertEquals(1, cache.evictionsMeter.count()) + assertEquals(9, cache.totalPartitions) + assertEquals(2, cache.size) + assertEquals(1, cache.evictionsMeter.count) cache.remove(id3) assertCacheContains(cache, id2) - assertEquals(1, cache.size()) - assertEquals(1, cache.evictionsMeter.count()) - assertEquals(4, cache.totalPartitions()) - val iter = session2.partitionMap.iterator() + assertEquals(1, cache.size) + assertEquals(1, cache.evictionsMeter.count) + assertEquals(4, cache.totalPartitions) + val iter = session2.partitionMap.iterator iter.next() iter.remove() - assertEquals(3, session2.size()) + assertEquals(3, session2.size) assertEquals(4, session2.cachedSize) cache.touch(session2, session2.lastUsedMs) - assertEquals(3, cache.totalPartitions()) + assertEquals(3, cache.totalPartitions) } val EMPTY_PART_LIST = Collections.unmodifiableList(new util.ArrayList[TopicPartition]()) @@ -201,25 +201,34 @@ class FetchSessionTest { assertEquals(Errors.INVALID_FETCH_SESSION_EPOCH, context6.updateAndGenerateResponseData(respData2).error()) + // Test generating a throttled response for the incremental fetch session + val reqData7 = new util.LinkedHashMap[TopicPartition, FetchRequest.PartitionData] + val context7 = fetchManager.newContext( + new JFetchMetadata(resp2.sessionId(), 2), reqData7, EMPTY_PART_LIST, false) + val resp7 = context7.getThrottledResponse(100) + assertEquals(Errors.NONE, resp7.error()) + assertEquals(resp2.sessionId(), resp7.sessionId()) + assertEquals(100, resp7.throttleTimeMs()) + // Close the incremental fetch session. val prevSessionId = resp5.sessionId var nextSessionId = prevSessionId do { - val reqData7 = new util.LinkedHashMap[TopicPartition, FetchRequest.PartitionData] - reqData7.put(new TopicPartition("bar", 0), new FetchRequest.PartitionData(0, 0, 100)) - reqData7.put(new TopicPartition("bar", 1), new FetchRequest.PartitionData(10, 0, 100)) - val context7 = fetchManager.newContext( - new JFetchMetadata(prevSessionId, FINAL_EPOCH), reqData7, EMPTY_PART_LIST, false) - assertEquals(classOf[SessionlessFetchContext], context7.getClass) - assertEquals(0, cache.size()) - val respData7 = new util.LinkedHashMap[TopicPartition, FetchResponse.PartitionData[Records]] - respData7.put(new TopicPartition("bar", 0), + val reqData8 = new util.LinkedHashMap[TopicPartition, FetchRequest.PartitionData] + reqData8.put(new TopicPartition("bar", 0), new FetchRequest.PartitionData(0, 0, 100)) + reqData8.put(new TopicPartition("bar", 1), new FetchRequest.PartitionData(10, 0, 100)) + val context8 = fetchManager.newContext( + new JFetchMetadata(prevSessionId, FINAL_EPOCH), reqData8, EMPTY_PART_LIST, false) + assertEquals(classOf[SessionlessFetchContext], context8.getClass) + assertEquals(0, cache.size) + val respData8 = new util.LinkedHashMap[TopicPartition, FetchResponse.PartitionData[Records]] + respData8.put(new TopicPartition("bar", 0), new FetchResponse.PartitionData(Errors.NONE, 100, 100, 100, null, null)) - respData7.put(new TopicPartition("bar", 1), + respData8.put(new TopicPartition("bar", 1), new FetchResponse.PartitionData(Errors.NONE, 100, 100, 100, null, null)) - val resp7 = context7.updateAndGenerateResponseData(respData7) - assertEquals(Errors.NONE, resp7.error()) - nextSessionId = resp7.sessionId() + val resp8 = context8.updateAndGenerateResponseData(respData8) + assertEquals(Errors.NONE, resp8.error) + nextSessionId = resp8.sessionId } while (nextSessionId == prevSessionId) } @@ -268,9 +277,9 @@ class FetchSessionTest { respData2.put(new TopicPartition("bar", 0), new FetchResponse.PartitionData( Errors.NONE, 10, 10, 10, null, null)) val resp2 = context2.updateAndGenerateResponseData(respData2) - assertEquals(Errors.NONE, resp2.error()) - assertEquals(1, resp2.responseData().size()) - assertTrue(resp2.sessionId() > 0) + assertEquals(Errors.NONE, resp2.error) + assertEquals(1, resp2.responseData.size) + assertTrue(resp2.sessionId > 0) } @Test @@ -291,9 +300,9 @@ class FetchSessionTest { respData1.put(new TopicPartition("foo", 1), new FetchResponse.PartitionData( Errors.NONE, 10, 10, 10, null, null)) val resp1 = context1.updateAndGenerateResponseData(respData1) - assertEquals(Errors.NONE, resp1.error()) + assertEquals(Errors.NONE, resp1.error) assertTrue(resp1.sessionId() != INVALID_SESSION_ID) - assertEquals(2, resp1.responseData().size()) + assertEquals(2, resp1.responseData.size) // Create an incremental fetch request that removes foo-0 and foo-1 // Verify that the previous fetch session was closed. @@ -302,12 +311,12 @@ class FetchSessionTest { removed2.add(new TopicPartition("foo", 0)) removed2.add(new TopicPartition("foo", 1)) val context2 = fetchManager.newContext( - new JFetchMetadata(resp1.sessionId(), 1), reqData2, removed2, false) + new JFetchMetadata(resp1.sessionId, 1), reqData2, removed2, false) assertEquals(classOf[SessionlessFetchContext], context2.getClass) val respData2 = new util.LinkedHashMap[TopicPartition, FetchResponse.PartitionData[Records]] val resp2 = context2.updateAndGenerateResponseData(respData2) - assertEquals(INVALID_SESSION_ID, resp2.sessionId()) + assertEquals(INVALID_SESSION_ID, resp2.sessionId) assertTrue(resp2.responseData().isEmpty) - assertEquals(0, cache.size()) + assertEquals(0, cache.size) } } diff --git a/core/src/test/scala/unit/kafka/server/ISRExpirationTest.scala b/core/src/test/scala/unit/kafka/server/ISRExpirationTest.scala index 8212ed680c5ba..cb914c43344ee 100644 --- a/core/src/test/scala/unit/kafka/server/ISRExpirationTest.scala +++ b/core/src/test/scala/unit/kafka/server/ISRExpirationTest.scala @@ -22,7 +22,7 @@ import java.util.concurrent.atomic.AtomicBoolean import kafka.cluster.{Partition, Replica} import kafka.log.Log -import kafka.server.epoch.LeaderEpochCache +import kafka.server.epoch.LeaderEpochFileCache import kafka.utils._ import org.apache.kafka.common.TopicPartition import org.apache.kafka.common.metrics.Metrics @@ -218,7 +218,7 @@ class IsrExpirationTest { private def logMock: Log = { val log = EasyMock.createMock(classOf[kafka.log.Log]) - val cache = EasyMock.createNiceMock(classOf[LeaderEpochCache]) + val cache = EasyMock.createNiceMock(classOf[LeaderEpochFileCache]) EasyMock.expect(log.dir).andReturn(TestUtils.tempDir()).anyTimes() EasyMock.expect(log.leaderEpochCache).andReturn(cache).anyTimes() EasyMock.expect(log.onHighWatermarkIncremented(0L)) diff --git a/core/src/test/scala/unit/kafka/server/KafkaConfigTest.scala b/core/src/test/scala/unit/kafka/server/KafkaConfigTest.scala index afb297d1ef495..0ee8d8152da4d 100755 --- a/core/src/test/scala/unit/kafka/server/KafkaConfigTest.scala +++ b/core/src/test/scala/unit/kafka/server/KafkaConfigTest.scala @@ -26,6 +26,7 @@ import kafka.utils.{CoreUtils, TestUtils} import org.apache.kafka.common.config.ConfigException import org.apache.kafka.common.metrics.Sensor import org.apache.kafka.common.network.ListenerName +import org.apache.kafka.common.record.Records import org.apache.kafka.common.security.auth.SecurityProtocol import org.junit.Assert._ import org.junit.Test @@ -590,7 +591,7 @@ class KafkaConfigTest { 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.MinMessageOverhead - 1) + case KafkaConfig.LogSegmentBytesProp => assertPropertyInvalid(getBaseProperties(), name, "not_a_number", Records.LOG_OVERHEAD - 1) case KafkaConfig.LogRollTimeMillisProp => assertPropertyInvalid(getBaseProperties(), name, "not_a_number", "0") case KafkaConfig.LogRollTimeHoursProp => assertPropertyInvalid(getBaseProperties(), name, "not_a_number", "0") diff --git a/core/src/test/scala/unit/kafka/server/LogOffsetTest.scala b/core/src/test/scala/unit/kafka/server/LogOffsetTest.scala index aa8236a267070..dd4f7e3391a82 100755 --- a/core/src/test/scala/unit/kafka/server/LogOffsetTest.scala +++ b/core/src/test/scala/unit/kafka/server/LogOffsetTest.scala @@ -21,71 +21,56 @@ import java.io.File import java.util.concurrent.atomic.AtomicInteger import java.util.{Properties, Random} -import kafka.api.{FetchRequestBuilder, OffsetRequest, PartitionOffsetRequestInfo} -import kafka.common.TopicAndPartition -import kafka.consumer.SimpleConsumer import kafka.log.{Log, LogSegment} -import kafka.utils.TestUtils._ -import kafka.utils._ -import kafka.zk.ZooKeeperTestHarness +import kafka.network.SocketServer +import kafka.utils.{MockTime, TestUtils} import org.apache.kafka.common.TopicPartition -import org.apache.kafka.common.protocol.Errors -import org.apache.kafka.common.utils.Time +import org.apache.kafka.common.protocol.{ApiKeys, Errors} +import org.apache.kafka.common.record.MemoryRecords +import org.apache.kafka.common.requests.{FetchRequest, FetchResponse, IsolationLevel, ListOffsetRequest, ListOffsetResponse} import org.easymock.{EasyMock, IAnswer} import org.junit.Assert._ -import org.junit.{After, Before, Test} - -class LogOffsetTest extends ZooKeeperTestHarness { - val random = new Random() - var logDir: File = null - var topicLogDir: File = null - var server: KafkaServer = null - var logSize: Int = 140 - var simpleConsumer: SimpleConsumer = null - var time: Time = new MockTime() - - @Before - override def setUp() { - super.setUp() - val config: Properties = createBrokerConfig(1) - config.put(KafkaConfig.LogMessageTimestampDifferenceMaxMsProp, Long.MaxValue.toString) - val logDirPath = config.getProperty("log.dir") - logDir = new File(logDirPath) - time = new MockTime() - server = TestUtils.createServer(KafkaConfig.fromProps(config), time) - simpleConsumer = new SimpleConsumer("localhost", TestUtils.boundPort(server), 1000000, 64*1024, "") - } +import org.junit.Test + +import scala.collection.JavaConverters._ + +class LogOffsetTest extends BaseRequestTest { + + private lazy val time = new MockTime + + protected override def numBrokers = 1 - @After - override def tearDown() { - simpleConsumer.close - TestUtils.shutdownServers(Seq(server)) - super.tearDown() + protected override def brokerTime(brokerId: Int) = time + + protected override def propertyOverrides(props: Properties): Unit = { + props.put("log.flush.interval.messages", "1") + props.put("num.partitions", "20") + props.put("log.retention.hours", "10") + props.put("log.retention.check.interval.ms", (5 * 1000 * 60).toString) + props.put("log.segment.bytes", "140") } @Test def testGetOffsetsForUnknownTopic() { - val topicAndPartition = TopicAndPartition("foo", 0) - val request = OffsetRequest( - Map(topicAndPartition -> PartitionOffsetRequestInfo(OffsetRequest.LatestTime, 10))) - val offsetResponse = simpleConsumer.getOffsetsBefore(request) - assertEquals(Errors.UNKNOWN_TOPIC_OR_PARTITION, - offsetResponse.partitionErrorAndOffsets(topicAndPartition).error) + val topicPartition = new TopicPartition("foo", 0) + val request = ListOffsetRequest.Builder.forConsumer(false, IsolationLevel.READ_UNCOMMITTED) + .setOffsetData(Map(topicPartition -> + new ListOffsetRequest.PartitionData(ListOffsetRequest.LATEST_TIMESTAMP, 10)).asJava).build(0) + val response = sendListOffsetsRequest(request) + assertEquals(Errors.UNKNOWN_TOPIC_OR_PARTITION, response.responseData.get(topicPartition).error) } @Test def testGetOffsetsAfterDeleteRecords() { - val topicPartition = "kafka-" + 0 - val topic = topicPartition.split("-").head - val part = Integer.valueOf(topicPartition.split("-").last).intValue + val topic = "kafka-" + val topicPartition = new TopicPartition(topic, 0) - // setup brokers in ZooKeeper as owners of partitions for this test adminZkClient.createTopic(topic, 1, 1) val logManager = server.getLogManager - waitUntilTrue(() => logManager.getLog(new TopicPartition(topic, part)).isDefined, + TestUtils.waitUntilTrue(() => logManager.getLog(topicPartition).isDefined, "Log for partition [topic,0] should be created") - val log = logManager.getLog(new TopicPartition(topic, part)).get + val log = logManager.getLog(topicPartition).get for (_ <- 0 until 20) log.appendAsLeader(TestUtils.singletonRecords(value = Integer.toString(42).getBytes()), leaderEpoch = 0) @@ -95,93 +80,87 @@ class LogOffsetTest extends ZooKeeperTestHarness { log.maybeIncrementLogStartOffset(3) log.deleteOldSegments() - val offsets = server.apis.fetchOffsets(logManager, new TopicPartition(topic, part), OffsetRequest.LatestTime, 15) + val offsets = server.apis.fetchOffsets(logManager, topicPartition, ListOffsetRequest.LATEST_TIMESTAMP, 15) assertEquals(Seq(20L, 18L, 16L, 14L, 12L, 10L, 8L, 6L, 4L, 3L), offsets) - waitUntilTrue(() => isLeaderLocalOnBroker(topic, part, server), "Leader should be elected") - val topicAndPartition = TopicAndPartition(topic, part) - val offsetRequest = OffsetRequest( - Map(topicAndPartition -> PartitionOffsetRequestInfo(OffsetRequest.LatestTime, 15)), - replicaId = 0) - val consumerOffsets = - simpleConsumer.getOffsetsBefore(offsetRequest).partitionErrorAndOffsets(topicAndPartition).offsets + TestUtils.waitUntilTrue(() => TestUtils.isLeaderLocalOnBroker(topic, topicPartition.partition, server), + "Leader should be elected") + val request = ListOffsetRequest.Builder.forReplica(0, 0) + .setOffsetData(Map(topicPartition -> + new ListOffsetRequest.PartitionData(ListOffsetRequest.LATEST_TIMESTAMP, 15)).asJava).build() + val consumerOffsets = sendListOffsetsRequest(request).responseData.get(topicPartition).offsets.asScala assertEquals(Seq(20L, 18L, 16L, 14L, 12L, 10L, 8L, 6L, 4L, 3L), consumerOffsets) } @Test def testGetOffsetsBeforeLatestTime() { - val topicPartition = "kafka-" + 0 - val topic = topicPartition.split("-").head - val part = Integer.valueOf(topicPartition.split("-").last).intValue + val topic = "kafka-" + val topicPartition = new TopicPartition(topic, 0) - // setup brokers in ZooKeeper as owners of partitions for this test adminZkClient.createTopic(topic, 1, 1) val logManager = server.getLogManager - waitUntilTrue(() => logManager.getLog(new TopicPartition(topic, part)).isDefined, - "Log for partition [topic,0] should be created") - val log = logManager.getLog(new TopicPartition(topic, part)).get + TestUtils.waitUntilTrue(() => logManager.getLog(topicPartition).isDefined, + s"Log for partition $topicPartition should be created") + val log = logManager.getLog(topicPartition).get for (_ <- 0 until 20) log.appendAsLeader(TestUtils.singletonRecords(value = Integer.toString(42).getBytes()), leaderEpoch = 0) log.flush() - val offsets = server.apis.fetchOffsets(logManager, new TopicPartition(topic, part), OffsetRequest.LatestTime, 15) + val offsets = server.apis.fetchOffsets(logManager, topicPartition, ListOffsetRequest.LATEST_TIMESTAMP, 15) assertEquals(Seq(20L, 18L, 16L, 14L, 12L, 10L, 8L, 6L, 4L, 2L, 0L), offsets) - waitUntilTrue(() => isLeaderLocalOnBroker(topic, part, server), "Leader should be elected") - val topicAndPartition = TopicAndPartition(topic, part) - val offsetRequest = OffsetRequest( - Map(topicAndPartition -> PartitionOffsetRequestInfo(OffsetRequest.LatestTime, 15)), - replicaId = 0) - val consumerOffsets = - simpleConsumer.getOffsetsBefore(offsetRequest).partitionErrorAndOffsets(topicAndPartition).offsets + TestUtils.waitUntilTrue(() => TestUtils.isLeaderLocalOnBroker(topic, topicPartition.partition, server), + "Leader should be elected") + val request = ListOffsetRequest.Builder.forReplica(0, 0) + .setOffsetData(Map(topicPartition -> + new ListOffsetRequest.PartitionData(ListOffsetRequest.LATEST_TIMESTAMP, 15)).asJava).build() + val consumerOffsets = sendListOffsetsRequest(request).responseData.get(topicPartition).offsets.asScala assertEquals(Seq(20L, 18L, 16L, 14L, 12L, 10L, 8L, 6L, 4L, 2L, 0L), consumerOffsets) // try to fetch using latest offset - val fetchResponse = simpleConsumer.fetch( - new FetchRequestBuilder().addFetch(topic, 0, consumerOffsets.head, 300 * 1024).build()) - assertFalse(fetchResponse.messageSet(topic, 0).iterator.hasNext) + val fetchRequest = FetchRequest.Builder.forConsumer(0, 1, + Map(topicPartition -> new FetchRequest.PartitionData(consumerOffsets.head, FetchRequest.INVALID_LOG_START_OFFSET, + 300 * 1024)).asJava).build() + val fetchResponse = sendFetchRequest(fetchRequest) + assertFalse(fetchResponse.responseData.get(topicPartition).records.batches.iterator.hasNext) } @Test def testEmptyLogsGetOffsets() { - val topicPartition = "kafka-" + random.nextInt(10) - val topicPartitionPath = TestUtils.tempDir().getAbsolutePath + "/" + topicPartition - topicLogDir = new File(topicPartitionPath) + val random = new Random + val topic = "kafka-" + val topicPartition = new TopicPartition(topic, random.nextInt(10)) + val topicPartitionPath = s"${TestUtils.tempDir().getAbsolutePath}/$topic-${topicPartition.partition}" + val topicLogDir = new File(topicPartitionPath) topicLogDir.mkdir() - val topic = topicPartition.split("-").head - - // setup brokers in ZooKeeper as owners of partitions for this test - createTopic(zkClient, topic, numPartitions = 1, replicationFactor = 1, servers = Seq(server)) + createTopic(topic, numPartitions = 1, replicationFactor = 1) var offsetChanged = false for (_ <- 1 to 14) { - val topicAndPartition = TopicAndPartition(topic, 0) - val offsetRequest = - OffsetRequest(Map(topicAndPartition -> PartitionOffsetRequestInfo(OffsetRequest.EarliestTime, 1))) - val consumerOffsets = - simpleConsumer.getOffsetsBefore(offsetRequest).partitionErrorAndOffsets(topicAndPartition).offsets - - if(consumerOffsets.head == 1) { + val topicPartition = new TopicPartition(topic, 0) + val request = ListOffsetRequest.Builder.forReplica(0, 0) + .setOffsetData(Map(topicPartition -> + new ListOffsetRequest.PartitionData(ListOffsetRequest.EARLIEST_TIMESTAMP, 1)).asJava).build() + val consumerOffsets = sendListOffsetsRequest(request).responseData.get(topicPartition).offsets.asScala + if (consumerOffsets.head == 1) offsetChanged = true - } } assertFalse(offsetChanged) } @Test def testGetOffsetsBeforeNow() { - val topicPartition = "kafka-" + random.nextInt(3) - val topic = topicPartition.split("-").head - val part = Integer.valueOf(topicPartition.split("-").last).intValue + val random = new Random + val topic = "kafka-" + val topicPartition = new TopicPartition(topic, random.nextInt(3)) - // setup brokers in ZooKeeper as owners of partitions for this test adminZkClient.createTopic(topic, 3, 1) val logManager = server.getLogManager - val log = logManager.getOrCreateLog(new TopicPartition(topic, part), logManager.initialDefaultConfig) + val log = logManager.getOrCreateLog(topicPartition, logManager.initialDefaultConfig) for (_ <- 0 until 20) log.appendAsLeader(TestUtils.singletonRecords(value = Integer.toString(42).getBytes()), leaderEpoch = 0) @@ -189,42 +168,42 @@ class LogOffsetTest extends ZooKeeperTestHarness { val now = time.milliseconds + 30000 // pretend it is the future to avoid race conditions with the fs - val offsets = server.apis.fetchOffsets(logManager, new TopicPartition(topic, part), now, 15) + val offsets = server.apis.fetchOffsets(logManager, topicPartition, now, 15) assertEquals(Seq(20L, 18L, 16L, 14L, 12L, 10L, 8L, 6L, 4L, 2L, 0L), offsets) - waitUntilTrue(() => isLeaderLocalOnBroker(topic, part, server), "Leader should be elected") - val topicAndPartition = TopicAndPartition(topic, part) - val offsetRequest = OffsetRequest(Map(topicAndPartition -> PartitionOffsetRequestInfo(now, 15)), replicaId = 0) - val consumerOffsets = - simpleConsumer.getOffsetsBefore(offsetRequest).partitionErrorAndOffsets(topicAndPartition).offsets + TestUtils.waitUntilTrue(() => TestUtils.isLeaderLocalOnBroker(topic, topicPartition.partition, server), + "Leader should be elected") + val request = ListOffsetRequest.Builder.forReplica(0, 0) + .setOffsetData(Map(topicPartition -> + new ListOffsetRequest.PartitionData(now, 15)).asJava).build() + val consumerOffsets = sendListOffsetsRequest(request).responseData.get(topicPartition).offsets.asScala assertEquals(Seq(20L, 18L, 16L, 14L, 12L, 10L, 8L, 6L, 4L, 2L, 0L), consumerOffsets) } @Test def testGetOffsetsBeforeEarliestTime() { - val topicPartition = "kafka-" + random.nextInt(3) - val topic = topicPartition.split("-").head - val part = Integer.valueOf(topicPartition.split("-").last).intValue + val random = new Random + val topic = "kafka-" + val topicPartition = new TopicPartition(topic, random.nextInt(3)) - // setup brokers in ZooKeeper as owners of partitions for this test adminZkClient.createTopic(topic, 3, 1) val logManager = server.getLogManager - val log = logManager.getOrCreateLog(new TopicPartition(topic, part), logManager.initialDefaultConfig) + val log = logManager.getOrCreateLog(topicPartition, logManager.initialDefaultConfig) for (_ <- 0 until 20) log.appendAsLeader(TestUtils.singletonRecords(value = Integer.toString(42).getBytes()), leaderEpoch = 0) log.flush() - val offsets = server.apis.fetchOffsets(logManager, new TopicPartition(topic, part), OffsetRequest.EarliestTime, 10) + val offsets = server.apis.fetchOffsets(logManager, topicPartition, ListOffsetRequest.EARLIEST_TIMESTAMP, 10) assertEquals(Seq(0L), offsets) - waitUntilTrue(() => isLeaderLocalOnBroker(topic, part, server), "Leader should be elected") - val topicAndPartition = TopicAndPartition(topic, part) - val offsetRequest = - OffsetRequest(Map(topicAndPartition -> PartitionOffsetRequestInfo(OffsetRequest.EarliestTime, 10))) - val consumerOffsets = - simpleConsumer.getOffsetsBefore(offsetRequest).partitionErrorAndOffsets(topicAndPartition).offsets + TestUtils.waitUntilTrue(() => TestUtils.isLeaderLocalOnBroker(topic, topicPartition.partition, server), + "Leader should be elected") + val request = ListOffsetRequest.Builder.forReplica(0, 0) + .setOffsetData(Map(topicPartition -> + new ListOffsetRequest.PartitionData(ListOffsetRequest.EARLIEST_TIMESTAMP, 10)).asJava).build() + val consumerOffsets = sendListOffsetsRequest(request).responseData.get(topicPartition).offsets.asScala assertEquals(Seq(0L), consumerOffsets) } @@ -264,19 +243,16 @@ class LogOffsetTest extends ZooKeeperTestHarness { server.apis.fetchOffsetsBefore(log, System.currentTimeMillis, 100) } - private def createBrokerConfig(nodeId: Int): Properties = { - val props = new Properties - props.put("broker.id", nodeId.toString) - props.put("port", TestUtils.RandomPort.toString()) - props.put("log.dir", TestUtils.tempDir().getAbsolutePath) - props.put("log.flush.interval.messages", "1") - props.put("enable.zookeeper", "false") - props.put("num.partitions", "20") - props.put("log.retention.hours", "10") - props.put("log.retention.check.interval.ms", (5*1000*60).toString) - props.put("log.segment.bytes", logSize.toString) - props.put("zookeeper.connect", zkConnect.toString) - props + private def server: KafkaServer = servers.head + + private def sendListOffsetsRequest(request: ListOffsetRequest, destination: Option[SocketServer] = None): ListOffsetResponse = { + val response = connectAndSend(request, ApiKeys.LIST_OFFSETS, destination = destination.getOrElse(anySocketServer)) + ListOffsetResponse.parse(response, request.version) + } + + private def sendFetchRequest(request: FetchRequest, destination: Option[SocketServer] = None): FetchResponse[MemoryRecords] = { + val response = connectAndSend(request, ApiKeys.FETCH, destination = destination.getOrElse(anySocketServer)) + FetchResponse.parse(response, request.version) } } diff --git a/core/src/test/scala/unit/kafka/server/LogRecoveryTest.scala b/core/src/test/scala/unit/kafka/server/LogRecoveryTest.scala index d4ac10c97dfbc..1bd15f7b537ca 100755 --- a/core/src/test/scala/unit/kafka/server/LogRecoveryTest.scala +++ b/core/src/test/scala/unit/kafka/server/LogRecoveryTest.scala @@ -65,7 +65,7 @@ class LogRecoveryTest extends ZooKeeperTestHarness { def updateProducer() = { if (producer != null) producer.close() - producer = TestUtils.createNewProducer( + producer = TestUtils.createProducer( TestUtils.getBrokerListStrFromServers(servers), retries = 5, keySerializer = new IntegerSerializer, @@ -143,6 +143,15 @@ class LogRecoveryTest extends ZooKeeperTestHarness { leader == 0 || leader == 1) assertEquals(hw, hwFile1.read.getOrElse(topicPartition, 0L)) + /** We plan to shutdown server2 and transfer the leadership to server1. + * With unclean leader election turned off, a prerequisite for the successful leadership transition + * is that server1 has caught up on the topicPartition, and has joined the ISR. + * In the line below, we wait until the condition is met before shutting down server2 + */ + waitUntilTrue(() => server2.replicaManager.getPartition(topicPartition).get.inSyncReplicas.size == 2, + "Server 1 is not able to join the ISR after restart") + + // since server 2 was never shut down, the hw value of 30 is probably not checkpointed to disk yet server2.shutdown() assertEquals(hw, hwFile2.read.getOrElse(topicPartition, 0L)) diff --git a/core/src/test/scala/unit/kafka/server/MetadataCacheTest.scala b/core/src/test/scala/unit/kafka/server/MetadataCacheTest.scala index 0ee73659780e1..82c14ee58278f 100644 --- a/core/src/test/scala/unit/kafka/server/MetadataCacheTest.scala +++ b/core/src/test/scala/unit/kafka/server/MetadataCacheTest.scala @@ -110,7 +110,47 @@ class MetadataCacheTest { } @Test - def getTopicMetadataPartitionLeaderNotAvailable() { + def getTopicMetadataPartitionLeaderNotAvailable(): Unit = { + val securityProtocol = SecurityProtocol.PLAINTEXT + val listenerName = ListenerName.forSecurityProtocol(securityProtocol) + val brokers = Set(new Broker(0, Seq(new EndPoint("foo", 9092, securityProtocol, listenerName)).asJava, null)) + verifyTopicMetadataPartitionLeaderOrEndpointNotAvailable(brokers, listenerName, + leader = 1, Errors.LEADER_NOT_AVAILABLE, errorUnavailableListeners = false) + verifyTopicMetadataPartitionLeaderOrEndpointNotAvailable(brokers, listenerName, + leader = 1, Errors.LEADER_NOT_AVAILABLE, errorUnavailableListeners = true) + } + + @Test + def getTopicMetadataPartitionListenerNotAvailableOnLeader(): Unit = { + val plaintextListenerName = ListenerName.forSecurityProtocol(SecurityProtocol.PLAINTEXT) + val sslListenerName = ListenerName.forSecurityProtocol(SecurityProtocol.SSL) + val broker0Endpoints = Seq( + new EndPoint("host0", 9092, SecurityProtocol.PLAINTEXT, plaintextListenerName), + new EndPoint("host0", 9093, SecurityProtocol.SSL, sslListenerName)) + val broker1Endpoints = Seq(new EndPoint("host1", 9092, SecurityProtocol.PLAINTEXT, plaintextListenerName)) + val brokers = Set(new Broker(0, broker0Endpoints.asJava, null), new Broker(1, broker1Endpoints.asJava, null)) + verifyTopicMetadataPartitionLeaderOrEndpointNotAvailable(brokers, sslListenerName, + leader = 1, Errors.LISTENER_NOT_FOUND, errorUnavailableListeners = true) + } + + @Test + def getTopicMetadataPartitionListenerNotAvailableOnLeaderOldMetadataVersion(): Unit = { + val plaintextListenerName = ListenerName.forSecurityProtocol(SecurityProtocol.PLAINTEXT) + val sslListenerName = ListenerName.forSecurityProtocol(SecurityProtocol.SSL) + val broker0Endpoints = Seq( + new EndPoint("host0", 9092, SecurityProtocol.PLAINTEXT, plaintextListenerName), + new EndPoint("host0", 9093, SecurityProtocol.SSL, sslListenerName)) + val broker1Endpoints = Seq(new EndPoint("host1", 9092, SecurityProtocol.PLAINTEXT, plaintextListenerName)) + val brokers = Set(new Broker(0, broker0Endpoints.asJava, null), new Broker(1, broker1Endpoints.asJava, null)) + verifyTopicMetadataPartitionLeaderOrEndpointNotAvailable(brokers, sslListenerName, + leader = 1, Errors.LEADER_NOT_AVAILABLE, errorUnavailableListeners = false) + } + + private def verifyTopicMetadataPartitionLeaderOrEndpointNotAvailable(brokers: Set[Broker], + listenerName: ListenerName, + leader: Int, + expectedError: Errors, + errorUnavailableListeners: Boolean): Unit = { val topic = "topic" val cache = new MetadataCache(1) @@ -118,11 +158,7 @@ class MetadataCacheTest { val zkVersion = 3 val controllerId = 2 val controllerEpoch = 1 - val securityProtocol = SecurityProtocol.PLAINTEXT - val listenerName = ListenerName.forSecurityProtocol(securityProtocol) - val brokers = Set(new Broker(0, Seq(new EndPoint("foo", 9092, securityProtocol, listenerName)).asJava, null)) - val leader = 1 val leaderEpoch = 1 val partitionStates = Map( new TopicPartition(topic, 0) -> new UpdateMetadataRequest.PartitionState(controllerEpoch, leader, leaderEpoch, asList(0), zkVersion, asList(0), asList())) @@ -132,7 +168,7 @@ class MetadataCacheTest { partitionStates.asJava, brokers.asJava).build() cache.updateCache(15, updateMetadataRequest) - val topicMetadatas = cache.getTopicMetadata(Set(topic), listenerName) + val topicMetadatas = cache.getTopicMetadata(Set(topic), listenerName, errorUnavailableListeners = errorUnavailableListeners) assertEquals(1, topicMetadatas.size) val topicMetadata = topicMetadatas.head @@ -143,7 +179,7 @@ class MetadataCacheTest { val partitionMetadata = partitionMetadatas.get(0) assertEquals(0, partitionMetadata.partition) - assertEquals(Errors.LEADER_NOT_AVAILABLE, partitionMetadata.error) + assertEquals(expectedError, partitionMetadata.error) assertTrue(partitionMetadata.isr.isEmpty) assertEquals(1, partitionMetadata.replicas.size) assertEquals(0, partitionMetadata.replicas.get(0).id) diff --git a/core/src/test/scala/unit/kafka/server/MetadataRequestTest.scala b/core/src/test/scala/unit/kafka/server/MetadataRequestTest.scala index d4c3e7cfc64d5..6b613813a86c2 100644 --- a/core/src/test/scala/unit/kafka/server/MetadataRequestTest.scala +++ b/core/src/test/scala/unit/kafka/server/MetadataRequestTest.scala @@ -21,6 +21,7 @@ import java.util.Properties import kafka.network.SocketServer import kafka.utils.TestUtils +import org.apache.kafka.common.Node import org.apache.kafka.common.internals.Topic import org.apache.kafka.common.protocol.{ApiKeys, Errors} import org.apache.kafka.common.requests.{MetadataRequest, MetadataResponse} @@ -33,6 +34,7 @@ import scala.collection.JavaConverters._ class MetadataRequestTest extends BaseRequestTest { override def propertyOverrides(properties: Properties) { + properties.setProperty(KafkaConfig.DefaultReplicationFactorProp, "2") properties.setProperty(KafkaConfig.RackProp, s"rack/${properties.getProperty(KafkaConfig.BrokerIdProp)}") } @@ -144,6 +146,49 @@ class MetadataRequestTest extends BaseRequestTest { assertEquals(None, zkClient.getTopicPartitionCount(topic4)) } + @Test + def testAutoCreateTopicWithInvalidReplicationFactor(): Unit = { + // Shutdown all but one broker so that the number of brokers is less than the default replication factor + servers.tail.foreach(_.shutdown()) + servers.tail.foreach(_.awaitShutdown()) + + val topic1 = "testAutoCreateTopic" + val response1 = sendMetadataRequest(new MetadataRequest.Builder(Seq(topic1).asJava, true).build) + assertEquals(1, response1.topicMetadata.size) + val topicMetadata = response1.topicMetadata.asScala.head + assertEquals(Errors.INVALID_REPLICATION_FACTOR, topicMetadata.error) + assertEquals(topic1, topicMetadata.topic) + assertEquals(0, topicMetadata.partitionMetadata.size) + } + + @Test + def testAutoCreateOfCollidingTopics(): Unit = { + val topic1 = "testAutoCreate_Topic" + val topic2 = "testAutoCreate.Topic" + val response1 = sendMetadataRequest(new MetadataRequest.Builder(Seq(topic1, topic2).asJava, true).build) + assertEquals(2, response1.topicMetadata.size) + var topicMetadata1 = response1.topicMetadata.asScala.head + val topicMetadata2 = response1.topicMetadata.asScala.toSeq(1) + assertEquals(Errors.LEADER_NOT_AVAILABLE, topicMetadata1.error) + assertEquals(topic1, topicMetadata1.topic) + assertEquals(Errors.INVALID_TOPIC_EXCEPTION, topicMetadata2.error) + assertEquals(topic2, topicMetadata2.topic) + + TestUtils.waitUntilLeaderIsElectedOrChanged(zkClient, topic1, 0) + TestUtils.waitUntilMetadataIsPropagated(servers, topic1, 0) + + // retry the metadata for the first auto created topic + val response2 = sendMetadataRequest(new MetadataRequest.Builder(Seq(topic1).asJava, true).build) + topicMetadata1 = response2.topicMetadata.asScala.head + assertEquals(Errors.NONE, topicMetadata1.error) + assertEquals(Seq(Errors.NONE), topicMetadata1.partitionMetadata.asScala.map(_.error)) + assertEquals(1, topicMetadata1.partitionMetadata.size) + val partitionMetadata = topicMetadata1.partitionMetadata.asScala.head + assertEquals(0, partitionMetadata.partition) + assertEquals(2, partitionMetadata.replicas.size) + assertNotNull(partitionMetadata.leader) + } + @Test def testAllTopicsRequest() { // create some topics @@ -235,8 +280,80 @@ class MetadataRequestTest extends BaseRequestTest { assertEquals(s"Response should have $replicaCount replicas", replicaCount, v1PartitionMetadata.replicas.size) } + @Test + def testIsrAfterBrokerShutDownAndJoinsBack(): Unit = { + def checkIsr(servers: Seq[KafkaServer], topic: String): Unit = { + val activeBrokers = servers.filter(_.brokerState.currentState != NotRunning.state) + val expectedIsr = activeBrokers.map { broker => + new Node(broker.config.brokerId, "localhost", TestUtils.boundPort(broker), broker.config.rack.orNull) + }.sortBy(_.id) + + // Assert that topic metadata at new brokers is updated correctly + activeBrokers.foreach { broker => + var actualIsr: Seq[Node] = Seq.empty + TestUtils.waitUntilTrue(() => { + val metadataResponse = sendMetadataRequest(new MetadataRequest.Builder(Seq(topic).asJava, false).build, + Some(brokerSocketServer(broker.config.brokerId))) + val firstPartitionMetadata = metadataResponse.topicMetadata.asScala.headOption.flatMap(_.partitionMetadata.asScala.headOption) + actualIsr = firstPartitionMetadata.map { partitionMetadata => + partitionMetadata.isr.asScala.sortBy(_.id) + }.getOrElse(Seq.empty) + expectedIsr == actualIsr + }, s"Topic metadata not updated correctly in broker $broker\n" + + s"Expected ISR: $expectedIsr \n" + + s"Actual ISR : $actualIsr") + } + } + + val topic = "isr-after-broker-shutdown" + val replicaCount = 3 + createTopic(topic, 1, replicaCount) + + servers.last.shutdown() + servers.last.awaitShutdown() + servers.last.startup() + + checkIsr(servers, topic) + } + + @Test + def testAliveBrokersWithNoTopics(): Unit = { + def checkMetadata(servers: Seq[KafkaServer], expectedBrokersCount: Int): Unit = { + var controllerMetadataResponse: Option[MetadataResponse] = None + TestUtils.waitUntilTrue(() => { + val metadataResponse = sendMetadataRequest(MetadataRequest.Builder.allTopics.build, + Some(controllerSocketServer)) + controllerMetadataResponse = Some(metadataResponse) + metadataResponse.brokers.size == expectedBrokersCount + }, s"Expected $expectedBrokersCount brokers, but there are ${controllerMetadataResponse.get.brokers.size} " + + "according to the Controller") + + val brokersInController = controllerMetadataResponse.get.brokers.asScala.toSeq.sortBy(_.id) + + // Assert that metadata is propagated correctly + servers.filter(_.brokerState.currentState != NotRunning.state).foreach { broker => + TestUtils.waitUntilTrue(() => { + val metadataResponse = sendMetadataRequest(MetadataRequest.Builder.allTopics.build, + Some(brokerSocketServer(broker.config.brokerId))) + val brokers = metadataResponse.brokers.asScala.toSeq.sortBy(_.id) + val topicMetadata = metadataResponse.topicMetadata.asScala.toSeq.sortBy(_.topic) + brokersInController == brokers && metadataResponse.topicMetadata.asScala.toSeq.sortBy(_.topic) == topicMetadata + }, s"Topic metadata not updated correctly") + } + } + + val serverToShutdown = servers.filterNot(_.kafkaController.isActive).last + serverToShutdown.shutdown() + serverToShutdown.awaitShutdown() + checkMetadata(servers, servers.size - 1) + + serverToShutdown.startup() + checkMetadata(servers, servers.size) + } + private def sendMetadataRequest(request: MetadataRequest, destination: Option[SocketServer] = None): MetadataResponse = { val response = connectAndSend(request, ApiKeys.METADATA, destination = destination.getOrElse(anySocketServer)) MetadataResponse.parse(response, request.version) } + } diff --git a/core/src/test/scala/unit/kafka/server/OffsetCommitTest.scala b/core/src/test/scala/unit/kafka/server/OffsetCommitTest.scala deleted file mode 100755 index 3e8a535946814..0000000000000 --- a/core/src/test/scala/unit/kafka/server/OffsetCommitTest.scala +++ /dev/null @@ -1,331 +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.api.{GroupCoordinatorRequest, OffsetCommitRequest, OffsetFetchRequest} -import kafka.consumer.SimpleConsumer -import kafka.common.{OffsetAndMetadata, OffsetMetadata, OffsetMetadataAndError, TopicAndPartition} -import kafka.utils._ -import kafka.utils.TestUtils._ -import kafka.zk.ZooKeeperTestHarness -import org.apache.kafka.common.protocol.Errors -import org.apache.kafka.common.utils.Time -import org.junit.{After, Before, Test} -import org.junit.Assert._ -import java.util.Properties -import java.io.File - -import scala.util.Random -import scala.collection._ - -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 - var simpleConsumer: SimpleConsumer = null - - @Before - override def setUp() { - super.setUp() - val config: Properties = createBrokerConfig(1, zkConnect, enableDeleteTopic = true) - config.setProperty(KafkaConfig.OffsetsTopicReplicationFactorProp, "1") - config.setProperty(KafkaConfig.OffsetsRetentionCheckIntervalMsProp, retentionCheckInterval.toString) - val logDirPath = config.getProperty("log.dir") - logDir = new File(logDirPath) - server = TestUtils.createServer(KafkaConfig.fromProps(config), Time.SYSTEM) - simpleConsumer = new SimpleConsumer("localhost", TestUtils.boundPort(server), 1000000, 64*1024, "test-client") - val consumerMetadataRequest = GroupCoordinatorRequest(group) - Stream.continually { - val consumerMetadataResponse = simpleConsumer.send(consumerMetadataRequest) - consumerMetadataResponse.coordinatorOpt.isDefined - }.dropWhile(success => { - if (!success) Thread.sleep(1000) - !success - }) - } - - @After - override def tearDown() { - simpleConsumer.close - TestUtils.shutdownServers(Seq(server)) - super.tearDown() - } - - @Test - def testUpdateOffsets() { - val topic = "topic" - - // Commit an offset - val topicAndPartition = TopicAndPartition(topic, 0) - val expectedReplicaAssignment = Map(0 -> List(1)) - // create the topic - createTopic(zkClient, topic, partitionReplicaAssignment = expectedReplicaAssignment, servers = Seq(server)) - - val commitRequest = OffsetCommitRequest(group, immutable.Map(topicAndPartition -> OffsetAndMetadata(offset = 42L))) - val commitResponse = simpleConsumer.commitOffsets(commitRequest) - - assertEquals(Errors.NONE, commitResponse.commitStatus.get(topicAndPartition).get) - - // Fetch it and verify - val fetchRequest = OffsetFetchRequest(group, Seq(topicAndPartition)) - val fetchResponse = simpleConsumer.fetchOffsets(fetchRequest) - - assertEquals(Errors.NONE, fetchResponse.requestInfo.get(topicAndPartition).get.error) - assertEquals(OffsetMetadata.NoMetadata, fetchResponse.requestInfo.get(topicAndPartition).get.metadata) - assertEquals(42L, fetchResponse.requestInfo.get(topicAndPartition).get.offset) - - // Commit a new offset - val commitRequest1 = OffsetCommitRequest(group, immutable.Map(topicAndPartition -> OffsetAndMetadata( - offset=100L, - metadata="some metadata" - ))) - val commitResponse1 = simpleConsumer.commitOffsets(commitRequest1) - - assertEquals(Errors.NONE, commitResponse1.commitStatus.get(topicAndPartition).get) - - // Fetch it and verify - val fetchRequest1 = OffsetFetchRequest(group, Seq(topicAndPartition)) - val fetchResponse1 = simpleConsumer.fetchOffsets(fetchRequest1) - - assertEquals(Errors.NONE, fetchResponse1.requestInfo.get(topicAndPartition).get.error) - 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 - def testCommitAndFetchOffsets() { - val topic1 = "topic-1" - val topic2 = "topic-2" - val topic3 = "topic-3" - 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"), - TopicAndPartition(topic2, 0) -> OffsetAndMetadata(offset=43L, metadata="metadata two"), - TopicAndPartition(topic3, 0) -> OffsetAndMetadata(offset=44L, metadata="metadata three"), - TopicAndPartition(topic2, 1) -> OffsetAndMetadata(offset=45L) - )) - val commitResponse = simpleConsumer.commitOffsets(commitRequest) - assertEquals(Errors.NONE, commitResponse.commitStatus.get(TopicAndPartition(topic1, 0)).get) - assertEquals(Errors.NONE, commitResponse.commitStatus.get(TopicAndPartition(topic2, 0)).get) - assertEquals(Errors.NONE, commitResponse.commitStatus.get(TopicAndPartition(topic3, 0)).get) - assertEquals(Errors.NONE, commitResponse.commitStatus.get(TopicAndPartition(topic2, 1)).get) - - val fetchRequest = OffsetFetchRequest(group, Seq( - TopicAndPartition(topic1, 0), - TopicAndPartition(topic2, 0), - TopicAndPartition(topic3, 0), - TopicAndPartition(topic2, 1), - TopicAndPartition(topic3, 1), // An unknown partition - TopicAndPartition(topic4, 0), // An unused topic - TopicAndPartition(topic5, 0) // An unknown topic - )) - val fetchResponse = simpleConsumer.fetchOffsets(fetchRequest) - - assertEquals(Errors.NONE, fetchResponse.requestInfo.get(TopicAndPartition(topic1, 0)).get.error) - - assertEquals(Errors.NONE, fetchResponse.requestInfo.get(TopicAndPartition(topic2, 0)).get.error) - assertEquals(Errors.NONE, fetchResponse.requestInfo.get(TopicAndPartition(topic2, 1)).get.error) - - assertEquals(Errors.NONE, fetchResponse.requestInfo.get(TopicAndPartition(topic3, 0)).get.error) - assertEquals(Errors.NONE, fetchResponse.requestInfo.get(TopicAndPartition(topic3, 1)).get.error) - assertEquals(OffsetMetadataAndError.NoOffset, fetchResponse.requestInfo.get(TopicAndPartition(topic3, 1)).get) - - assertEquals(Errors.NONE, fetchResponse.requestInfo.get(TopicAndPartition(topic4, 0)).get.error) - assertEquals(OffsetMetadataAndError.NoOffset, fetchResponse.requestInfo.get(TopicAndPartition(topic4, 0)).get) - - assertEquals(Errors.NONE, fetchResponse.requestInfo.get(TopicAndPartition(topic5, 0)).get.error) - assertEquals(OffsetMetadataAndError.NoOffset, fetchResponse.requestInfo.get(TopicAndPartition(topic5, 0)).get) - - assertEquals("metadata one", fetchResponse.requestInfo.get(TopicAndPartition(topic1, 0)).get.metadata) - assertEquals("metadata two", fetchResponse.requestInfo.get(TopicAndPartition(topic2, 0)).get.metadata) - assertEquals("metadata three", fetchResponse.requestInfo.get(TopicAndPartition(topic3, 0)).get.metadata) - - assertEquals(OffsetMetadata.NoMetadata, fetchResponse.requestInfo.get(TopicAndPartition(topic2, 1)).get.metadata) - assertEquals(OffsetMetadata.NoMetadata, fetchResponse.requestInfo.get(TopicAndPartition(topic3, 1)).get.metadata) - assertEquals(OffsetMetadata.NoMetadata, fetchResponse.requestInfo.get(TopicAndPartition(topic4, 0)).get.metadata) - assertEquals(OffsetMetadata.NoMetadata, fetchResponse.requestInfo.get(TopicAndPartition(topic5, 0)).get.metadata) - - assertEquals(42L, fetchResponse.requestInfo.get(TopicAndPartition(topic1, 0)).get.offset) - assertEquals(43L, fetchResponse.requestInfo.get(TopicAndPartition(topic2, 0)).get.offset) - assertEquals(44L, fetchResponse.requestInfo.get(TopicAndPartition(topic3, 0)).get.offset) - assertEquals(45L, fetchResponse.requestInfo.get(TopicAndPartition(topic2, 1)).get.offset) - - assertEquals(OffsetMetadata.InvalidOffset, fetchResponse.requestInfo.get(TopicAndPartition(topic3, 1)).get.offset) - assertEquals(OffsetMetadata.InvalidOffset, fetchResponse.requestInfo.get(TopicAndPartition(topic4, 0)).get.offset) - assertEquals(OffsetMetadata.InvalidOffset, fetchResponse.requestInfo.get(TopicAndPartition(topic5, 0)).get.offset) - } - - @Test - def testLargeMetadataPayload() { - val topicAndPartition = TopicAndPartition("large-metadata", 0) - val expectedReplicaAssignment = Map(0 -> List(1)) - createTopic(zkClient, topicAndPartition.topic, partitionReplicaAssignment = expectedReplicaAssignment, - servers = Seq(server)) - - val commitRequest = OffsetCommitRequest("test-group", immutable.Map(topicAndPartition -> OffsetAndMetadata( - offset=42L, - metadata=random.nextString(server.config.offsetMetadataMaxSize) - ))) - val commitResponse = simpleConsumer.commitOffsets(commitRequest) - - assertEquals(Errors.NONE, commitResponse.commitStatus.get(topicAndPartition).get) - - val commitRequest1 = OffsetCommitRequest(group, immutable.Map(topicAndPartition -> OffsetAndMetadata( - offset=42L, - metadata=random.nextString(server.config.offsetMetadataMaxSize + 1) - ))) - val commitResponse1 = simpleConsumer.commitOffsets(commitRequest1) - - assertEquals(Errors.OFFSET_METADATA_TOO_LARGE, 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(Errors.NONE, 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 - val offsetFetchReq = OffsetFetchRequest(group, Seq(TopicAndPartition(topic, 0)), versionId = 0) - val offsetFetchResp = simpleConsumer.fetchOffsets(offsetFetchReq) - assertEquals(1L, offsetFetchResp.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(Errors.NONE, 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 - seven + a bit days - // committed offset should expire - val commitRequest2 = OffsetCommitRequest( - groupId = group, - requestInfo = immutable.Map(topicPartition -> OffsetAndMetadata(3L, "metadata", Time.SYSTEM.milliseconds - (Defaults.OffsetsRetentionMinutes + 1) * 60 * 1000L)), - versionId = 1 - ) - assertEquals(Errors.NONE, 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(Errors.NONE, 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(Errors.NONE, 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 commitRequest = OffsetCommitRequest(group, immutable.Map( - TopicAndPartition(topic1, 0) -> OffsetAndMetadata(offset=42L), - TopicAndPartition(topic2, 0) -> OffsetAndMetadata(offset=42L) - )) - val commitResponse = simpleConsumer.commitOffsets(commitRequest) - - assertEquals(Errors.UNKNOWN_TOPIC_OR_PARTITION, commitResponse.commitStatus.get(TopicAndPartition(topic1, 0)).get) - assertEquals(Errors.NONE, commitResponse.commitStatus.get(TopicAndPartition(topic2, 0)).get) - } - - @Test - def testOffsetsDeleteAfterTopicDeletion() { - // set up topic partition - val topic = "topic" - val topicPartition = TopicAndPartition(topic, 0) - createTopic(zkClient, topic, servers = Seq(server), numPartitions = 1) - - val commitRequest = OffsetCommitRequest(group, immutable.Map(topicPartition -> OffsetAndMetadata(offset = 42L))) - val commitResponse = simpleConsumer.commitOffsets(commitRequest) - - assertEquals(Errors.NONE, commitResponse.commitStatus.get(topicPartition).get) - - // start topic deletion - adminZkClient.deleteTopic(topic) - TestUtils.verifyTopicDeletion(zkClient, topic, 1, Seq(server)) - Thread.sleep(retentionCheckInterval * 2) - - // check if offsets deleted - val fetchRequest = OffsetFetchRequest(group, Seq(TopicAndPartition(topic, 0))) - val offsetMetadataAndErrorMap = simpleConsumer.fetchOffsets(fetchRequest) - val offsetMetadataAndError = offsetMetadataAndErrorMap.requestInfo(topicPartition) - assertEquals(OffsetMetadataAndError.NoOffset, offsetMetadataAndError) - } - -} diff --git a/core/src/test/scala/unit/kafka/server/ReplicaAlterLogDirsThreadTest.scala b/core/src/test/scala/unit/kafka/server/ReplicaAlterLogDirsThreadTest.scala index 29a1c9f069729..bdf83a53be671 100644 --- a/core/src/test/scala/unit/kafka/server/ReplicaAlterLogDirsThreadTest.scala +++ b/core/src/test/scala/unit/kafka/server/ReplicaAlterLogDirsThreadTest.scala @@ -21,7 +21,7 @@ import kafka.api.Request import kafka.cluster.{BrokerEndPoint, Partition, Replica} import kafka.log.LogManager import kafka.server.AbstractFetcherThread.ResultWithPartitions -import kafka.server.epoch.LeaderEpochCache +import kafka.server.epoch.LeaderEpochFileCache import kafka.utils.{DelayedItem, TestUtils} import org.apache.kafka.common.TopicPartition import org.apache.kafka.common.errors.{KafkaStorageException, ReplicaNotAvailableException} @@ -46,7 +46,7 @@ class ReplicaAlterLogDirsThreadTest { val config = KafkaConfig.fromProps(TestUtils.createBrokerConfig(1, "localhost:1234")) //Setup all dependencies - val leaderEpochs = createNiceMock(classOf[LeaderEpochCache]) + val leaderEpochs = createNiceMock(classOf[LeaderEpochFileCache]) val replica = createNiceMock(classOf[Replica]) val futureReplica = createNiceMock(classOf[Replica]) val partition = createMock(classOf[Partition]) @@ -57,7 +57,7 @@ class ReplicaAlterLogDirsThreadTest { //Stubs expect(replica.epochs).andReturn(Some(leaderEpochs)).anyTimes() - expect(leaderEpochs.endOffsetFor(leaderEpoch)).andReturn((leaderEpoch, leo)).anyTimes() + expect(replica.endOffsetFor(leaderEpoch)).andReturn((leaderEpoch, leo)).anyTimes() stub(replica, replica, futureReplica, partition, replicaManager) replay(leaderEpochs, replicaManager, replica) @@ -87,7 +87,7 @@ class ReplicaAlterLogDirsThreadTest { val config = KafkaConfig.fromProps(TestUtils.createBrokerConfig(1, "localhost:1234")) //Setup all dependencies - val leaderEpochs = createNiceMock(classOf[LeaderEpochCache]) + val leaderEpochs = createNiceMock(classOf[LeaderEpochFileCache]) val replica = createNiceMock(classOf[Replica]) val partition = createMock(classOf[Partition]) val replicaManager = createMock(classOf[ReplicaManager]) @@ -97,7 +97,7 @@ class ReplicaAlterLogDirsThreadTest { //Stubs expect(replica.epochs).andReturn(Some(leaderEpochs)).anyTimes() - expect(leaderEpochs.endOffsetFor(leaderEpoch)).andReturn((leaderEpoch, leo)).anyTimes() + expect(replica.endOffsetFor(leaderEpoch)).andReturn((leaderEpoch, leo)).anyTimes() expect(replicaManager.getReplicaOrException(t1p0)).andReturn(replica).anyTimes() expect(replicaManager.getPartition(t1p0)).andReturn(Some(partition)).anyTimes() expect(replicaManager.getReplicaOrException(t1p1)).andThrow(new KafkaStorageException).once() @@ -133,9 +133,9 @@ class ReplicaAlterLogDirsThreadTest { // Setup all the dependencies val config = KafkaConfig.fromProps(TestUtils.createBrokerConfig(1, "localhost:1234")) val quotaManager = createNiceMock(classOf[ReplicationQuotaManager]) - val leaderEpochsT1p0 = createMock(classOf[LeaderEpochCache]) - val leaderEpochsT1p1 = createMock(classOf[LeaderEpochCache]) - val futureReplicaLeaderEpochs = createMock(classOf[LeaderEpochCache]) + val leaderEpochsT1p0 = createMock(classOf[LeaderEpochFileCache]) + val leaderEpochsT1p1 = createMock(classOf[LeaderEpochFileCache]) + val futureReplicaLeaderEpochs = createMock(classOf[LeaderEpochFileCache]) val logManager = createMock(classOf[LogManager]) val replicaT1p0 = createNiceMock(classOf[Replica]) val replicaT1p1 = createNiceMock(classOf[Replica]) @@ -157,9 +157,9 @@ class ReplicaAlterLogDirsThreadTest { expect(futureReplica.epochs).andReturn(Some(futureReplicaLeaderEpochs)).anyTimes() expect(futureReplica.logEndOffset).andReturn(new LogOffsetMetadata(futureReplicaLEO)).anyTimes() expect(futureReplicaLeaderEpochs.latestEpoch).andReturn(leaderEpoch).anyTimes() - expect(leaderEpochsT1p0.endOffsetFor(leaderEpoch)).andReturn((leaderEpoch, replicaT1p0LEO)).anyTimes() - expect(leaderEpochsT1p1.endOffsetFor(leaderEpoch)).andReturn((leaderEpoch, replicaT1p1LEO)).anyTimes() - expect(futureReplicaLeaderEpochs.endOffsetFor(leaderEpoch)).andReturn((leaderEpoch, futureReplicaLEO)).anyTimes() + expect(replicaT1p0.endOffsetFor(leaderEpoch)).andReturn((leaderEpoch, replicaT1p0LEO)).anyTimes() + expect(replicaT1p1.endOffsetFor(leaderEpoch)).andReturn((leaderEpoch, replicaT1p1LEO)).anyTimes() + expect(futureReplica.endOffsetFor(leaderEpoch)).andReturn((leaderEpoch, futureReplicaLEO)).anyTimes() expect(replicaManager.logManager).andReturn(logManager).anyTimes() stubWithFetchMessages(replicaT1p0, replicaT1p1, futureReplica, partition, replicaManager, responseCallback) @@ -195,8 +195,8 @@ class ReplicaAlterLogDirsThreadTest { // Setup all the dependencies val config = KafkaConfig.fromProps(TestUtils.createBrokerConfig(1, "localhost:1234")) val quotaManager = createNiceMock(classOf[ReplicationQuotaManager]) - val leaderEpochs = createMock(classOf[LeaderEpochCache]) - val futureReplicaLeaderEpochs = createMock(classOf[LeaderEpochCache]) + val leaderEpochs = createMock(classOf[LeaderEpochFileCache]) + val futureReplicaLeaderEpochs = createMock(classOf[LeaderEpochFileCache]) val logManager = createMock(classOf[LogManager]) val replica = createNiceMock(classOf[Replica]) // one future replica mock because our mocking methods return same values for both future replicas @@ -220,12 +220,12 @@ class ReplicaAlterLogDirsThreadTest { expect(futureReplicaLeaderEpochs.latestEpoch).andReturn(leaderEpoch - 2).once() // leader replica truncated and fetched new offsets with new leader epoch - expect(leaderEpochs.endOffsetFor(leaderEpoch)).andReturn((leaderEpoch - 1, replicaLEO)).anyTimes() + expect(replica.endOffsetFor(leaderEpoch)).andReturn((leaderEpoch - 1, replicaLEO)).anyTimes() // but future replica does not know about this leader epoch, so returns a smaller leader epoch - expect(futureReplicaLeaderEpochs.endOffsetFor(leaderEpoch - 1)).andReturn((leaderEpoch - 2, futureReplicaLEO)).anyTimes() + expect(futureReplica.endOffsetFor(leaderEpoch - 1)).andReturn((leaderEpoch - 2, futureReplicaLEO)).anyTimes() // finally, the leader replica knows about the leader epoch and returns end offset - expect(leaderEpochs.endOffsetFor(leaderEpoch - 2)).andReturn((leaderEpoch - 2, replicaEpochEndOffset)).anyTimes() - expect(futureReplicaLeaderEpochs.endOffsetFor(leaderEpoch - 2)).andReturn((leaderEpoch - 2, futureReplicaEpochEndOffset)).anyTimes() + expect(replica.endOffsetFor(leaderEpoch - 2)).andReturn((leaderEpoch - 2, replicaEpochEndOffset)).anyTimes() + expect(futureReplica.endOffsetFor(leaderEpoch - 2)).andReturn((leaderEpoch - 2, futureReplicaEpochEndOffset)).anyTimes() expect(replicaManager.logManager).andReturn(logManager).anyTimes() stubWithFetchMessages(replica, replica, futureReplica, partition, replicaManager, responseCallback) @@ -265,8 +265,8 @@ class ReplicaAlterLogDirsThreadTest { val logManager = createMock(classOf[LogManager]) val replica = createNiceMock(classOf[Replica]) val futureReplica = createNiceMock(classOf[Replica]) - val leaderEpochs = createNiceMock(classOf[LeaderEpochCache]) - val futureReplicaLeaderEpochs = createMock(classOf[LeaderEpochCache]) + val leaderEpochs = createNiceMock(classOf[LeaderEpochFileCache]) + val futureReplicaLeaderEpochs = createMock(classOf[LeaderEpochFileCache]) val partition = createMock(classOf[Partition]) val replicaManager = createMock(classOf[ReplicaManager]) val responseCallback: Capture[Seq[(TopicPartition, FetchPartitionData)] => Unit] = EasyMock.newCapture() @@ -286,7 +286,7 @@ class ReplicaAlterLogDirsThreadTest { // since UNDEFINED_EPOCH is -1 which will be lower than any valid leader epoch, the method // will return UNDEFINED_EPOCH_OFFSET if requested epoch is lower than the first epoch cached - expect(leaderEpochs.endOffsetFor(UNDEFINED_EPOCH)).andReturn((UNDEFINED_EPOCH, UNDEFINED_EPOCH_OFFSET)).anyTimes() + expect(replica.endOffsetFor(UNDEFINED_EPOCH)).andReturn((UNDEFINED_EPOCH, UNDEFINED_EPOCH_OFFSET)).anyTimes() stubWithFetchMessages(replica, replica, futureReplica, partition, replicaManager, responseCallback) replay(replicaManager, logManager, quotaManager, leaderEpochs, futureReplicaLeaderEpochs, replica, futureReplica, partition) @@ -319,8 +319,8 @@ class ReplicaAlterLogDirsThreadTest { // Setup all the dependencies val config = KafkaConfig.fromProps(TestUtils.createBrokerConfig(1, "localhost:1234")) val quotaManager = createNiceMock(classOf[kafka.server.ReplicationQuotaManager]) - val leaderEpochs = createNiceMock(classOf[LeaderEpochCache]) - val futureReplicaLeaderEpochs = createMock(classOf[LeaderEpochCache]) + val leaderEpochs = createNiceMock(classOf[LeaderEpochFileCache]) + val futureReplicaLeaderEpochs = createMock(classOf[LeaderEpochFileCache]) val logManager = createMock(classOf[kafka.log.LogManager]) val replica = createNiceMock(classOf[Replica]) val futureReplica = createNiceMock(classOf[Replica]) @@ -338,8 +338,8 @@ class ReplicaAlterLogDirsThreadTest { expect(futureReplica.epochs).andReturn(Some(futureReplicaLeaderEpochs)).anyTimes() expect(futureReplicaLeaderEpochs.latestEpoch).andReturn(futureReplicaLeaderEpoch).anyTimes() - expect(leaderEpochs.endOffsetFor(futureReplicaLeaderEpoch)).andReturn((futureReplicaLeaderEpoch, replicaLEO)).anyTimes() - expect(futureReplicaLeaderEpochs.endOffsetFor(futureReplicaLeaderEpoch)).andReturn((futureReplicaLeaderEpoch, futureReplicaLEO)).anyTimes() + expect(replica.endOffsetFor(futureReplicaLeaderEpoch)).andReturn((futureReplicaLeaderEpoch, replicaLEO)).anyTimes() + expect(futureReplica.endOffsetFor(futureReplicaLeaderEpoch)).andReturn((futureReplicaLeaderEpoch, futureReplicaLEO)).anyTimes() expect(futureReplica.logEndOffset).andReturn(new LogOffsetMetadata(futureReplicaLEO)).anyTimes() expect(replicaManager.getReplica(t1p0)).andReturn(Some(replica)).anyTimes() @@ -401,8 +401,8 @@ class ReplicaAlterLogDirsThreadTest { //Setup all dependencies val config = KafkaConfig.fromProps(TestUtils.createBrokerConfig(1, "localhost:1234")) val quotaManager = createNiceMock(classOf[ReplicationQuotaManager]) - val leaderEpochs = createNiceMock(classOf[LeaderEpochCache]) - val futureReplicaLeaderEpochs = createMock(classOf[LeaderEpochCache]) + val leaderEpochs = createNiceMock(classOf[LeaderEpochFileCache]) + val futureReplicaLeaderEpochs = createMock(classOf[LeaderEpochFileCache]) val logManager = createMock(classOf[LogManager]) val replica = createNiceMock(classOf[Replica]) val futureReplica = createNiceMock(classOf[Replica]) @@ -420,9 +420,9 @@ class ReplicaAlterLogDirsThreadTest { expect(futureReplica.epochs).andReturn(Some(futureReplicaLeaderEpochs)).anyTimes() expect(futureReplica.logEndOffset).andReturn(new LogOffsetMetadata(futureReplicaLEO)).anyTimes() - expect(futureReplicaLeaderEpochs.latestEpoch).andReturn(leaderEpoch) - expect(leaderEpochs.endOffsetFor(leaderEpoch)).andReturn((leaderEpoch, replicaLEO)) - expect(futureReplicaLeaderEpochs.endOffsetFor(leaderEpoch)).andReturn((leaderEpoch, futureReplicaLEO)) + expect(futureReplicaLeaderEpochs.latestEpoch).andReturn(leaderEpoch).anyTimes() + expect(replica.endOffsetFor(leaderEpoch)).andReturn((leaderEpoch, replicaLEO)) + expect(futureReplica.endOffsetFor(leaderEpoch)).andReturn((leaderEpoch, futureReplicaLEO)) expect(replicaManager.logManager).andReturn(logManager).anyTimes() stubWithFetchMessages(replica, replica, futureReplica, partition, replicaManager, responseCallback) @@ -443,7 +443,7 @@ class ReplicaAlterLogDirsThreadTest { // loop few times (0 to 3).foreach { _ => thread.doWork() - } + } //Assert that truncate to is called exactly once (despite more loops) verify(partition) diff --git a/core/src/test/scala/unit/kafka/server/ReplicaFetchTest.scala b/core/src/test/scala/unit/kafka/server/ReplicaFetchTest.scala index c478ba2dc3731..0dd22f14f6eb9 100644 --- a/core/src/test/scala/unit/kafka/server/ReplicaFetchTest.scala +++ b/core/src/test/scala/unit/kafka/server/ReplicaFetchTest.scala @@ -55,7 +55,7 @@ class ReplicaFetchTest extends ZooKeeperTestHarness { } // send test messages to leader - val producer = TestUtils.createNewProducer(TestUtils.getBrokerListStrFromServers(brokers), + val producer = TestUtils.createProducer(TestUtils.getBrokerListStrFromServers(brokers), retries = 5, keySerializer = new StringSerializer, valueSerializer = new StringSerializer) diff --git a/core/src/test/scala/unit/kafka/server/ReplicaFetcherThreadTest.scala b/core/src/test/scala/unit/kafka/server/ReplicaFetcherThreadTest.scala index ac5b7edc3c7f1..e79d2af9cca29 100644 --- a/core/src/test/scala/unit/kafka/server/ReplicaFetcherThreadTest.scala +++ b/core/src/test/scala/unit/kafka/server/ReplicaFetcherThreadTest.scala @@ -19,12 +19,14 @@ package kafka.server import kafka.cluster.{BrokerEndPoint, Replica} import kafka.log.LogManager import kafka.cluster.Partition -import kafka.server.epoch.LeaderEpochCache +import kafka.server.QuotaFactory.UnboundedQuota +import kafka.server.epoch.LeaderEpochFileCache import kafka.server.epoch.util.ReplicaFetcherMockBlockingSend import kafka.utils.TestUtils import org.apache.kafka.common.TopicPartition +import org.apache.kafka.common.internals.PartitionStates import org.apache.kafka.common.metrics.Metrics -import org.apache.kafka.common.protocol.Errors +import org.apache.kafka.common.protocol.{ApiKeys, Errors} import org.apache.kafka.common.protocol.Errors._ import org.apache.kafka.common.requests.EpochEndOffset import org.apache.kafka.common.requests.EpochEndOffset._ @@ -45,6 +47,25 @@ class ReplicaFetcherThreadTest { private val brokerEndPoint = new BrokerEndPoint(0, "localhost", 1000) + @Test + def shouldSendLatestRequestVersionsByDefault(): Unit = { + val props = TestUtils.createBrokerConfig(1, "localhost:1234") + val config = KafkaConfig.fromProps(props) + val thread = new ReplicaFetcherThread( + name = "bob", + fetcherId = 0, + sourceBroker = brokerEndPoint, + brokerConfig = config, + replicaMgr = null, + metrics = new Metrics(), + time = new SystemTime(), + quota = UnboundedQuota, + leaderEndpointBlockingSend = None) + assertEquals(ApiKeys.FETCH.latestVersion, thread.fetchRequestVersion) + assertEquals(ApiKeys.OFFSET_FOR_LEADER_EPOCH.latestVersion, thread.offsetForLeaderEpochRequestVersion) + assertEquals(ApiKeys.LIST_OFFSETS.latestVersion, thread.listOffsetRequestVersion) + } + @Test def shouldNotIssueLeaderEpochRequestIfInterbrokerVersionBelow11(): Unit = { val props = TestUtils.createBrokerConfig(1, "localhost:1234") @@ -72,6 +93,100 @@ class ReplicaFetcherThreadTest { assertEquals("results from leader epoch request should have undefined offset", expected, result) } + @Test + def shouldFetchLeaderEpochRequestIfLastEpochDefinedForSomePartitions(): Unit = { + val config = KafkaConfig.fromProps(TestUtils.createBrokerConfig(1, "localhost:1234")) + + //Setup all dependencies + val quota = createNiceMock(classOf[ReplicationQuotaManager]) + val leaderEpochs = createNiceMock(classOf[LeaderEpochFileCache]) + val logManager = createMock(classOf[LogManager]) + val replicaAlterLogDirsManager = createMock(classOf[ReplicaAlterLogDirsManager]) + val replica = createNiceMock(classOf[Replica]) + val partition = createMock(classOf[Partition]) + val replicaManager = createMock(classOf[ReplicaManager]) + + val leaderEpoch = 5 + + //Stubs + expect(replica.epochs).andReturn(Some(leaderEpochs)).anyTimes() + expect(replica.logEndOffset).andReturn(new LogOffsetMetadata(0)).anyTimes() + expect(replica.highWatermark).andReturn(new LogOffsetMetadata(0)).anyTimes() + expect(leaderEpochs.latestEpoch).andReturn(leaderEpoch).once() + expect(leaderEpochs.latestEpoch).andReturn(leaderEpoch).once() + expect(leaderEpochs.latestEpoch).andReturn(UNDEFINED_EPOCH).once() // t2p1 doesnt support epochs + expect(replica.endOffsetFor(leaderEpoch)).andReturn((leaderEpoch, 0)).anyTimes() + expect(replicaManager.logManager).andReturn(logManager).anyTimes() + expect(replicaManager.replicaAlterLogDirsManager).andReturn(replicaAlterLogDirsManager).anyTimes() + stub(replica, partition, replicaManager) + + //Expectations + expect(partition.truncateTo(anyLong(), anyBoolean())).once + + replay(leaderEpochs, replicaManager, logManager, quota, replica) + + //Define the offsets for the OffsetsForLeaderEpochResponse + val offsets = Map(t1p0 -> new EpochEndOffset(leaderEpoch, 1), + t1p1 -> new EpochEndOffset(leaderEpoch, 1), + t2p1 -> new EpochEndOffset(-1, 1)).asJava + + //Create the fetcher thread + val mockNetwork = new ReplicaFetcherMockBlockingSend(offsets, brokerEndPoint, new SystemTime()) + + val thread = new ReplicaFetcherThread("bob", 0, brokerEndPoint, config, replicaManager, new Metrics(), new SystemTime(), quota, Some(mockNetwork)) + + // topic 1 supports epoch, t2 doesn't + thread.addPartitions(Map(t1p0 -> 0, t1p1 -> 0, t2p1 -> 0)) + + assertPartitionStates(thread.partitionStates, shouldBeReadyForFetch = false, shouldBeTruncatingLog = true, shouldBeDelayed = false) + //Loop 1 + thread.doWork() + assertEquals(1, mockNetwork.epochFetchCount) + assertEquals(1, mockNetwork.fetchCount) + + assertPartitionStates(thread.partitionStates, shouldBeReadyForFetch = true, shouldBeTruncatingLog = false, shouldBeDelayed = false) + + //Loop 2 we should not fetch epochs + thread.doWork() + assertEquals(1, mockNetwork.epochFetchCount) + assertEquals(2, mockNetwork.fetchCount) + + assertPartitionStates(thread.partitionStates, shouldBeReadyForFetch = true, shouldBeTruncatingLog = false, shouldBeDelayed = false) + + //Loop 3 we should not fetch epochs + thread.doWork() + assertEquals(1, mockNetwork.epochFetchCount) + assertEquals(3, mockNetwork.fetchCount) + + assertPartitionStates(thread.partitionStates, shouldBeReadyForFetch = true, shouldBeTruncatingLog = false, shouldBeDelayed = false) + + //Assert that truncate to is called exactly once (despite two loops) + verify(logManager) + } + + /** + * Assert that all partitions' states are as expected + * + */ + def assertPartitionStates(states: PartitionStates[PartitionFetchState], shouldBeReadyForFetch: Boolean, + shouldBeTruncatingLog: Boolean, shouldBeDelayed: Boolean): Unit = { + for (tp <- List(t1p0, t1p1, t2p1)) { + assertEquals( + s"Partition $tp should${if (!shouldBeReadyForFetch) " NOT" else ""} be ready for fetching", + shouldBeReadyForFetch, states.stateValue(tp).isReadyForFetch) + + assertEquals( + s"Partition $tp should${if (!shouldBeTruncatingLog) " NOT" else ""} be truncating its log", + shouldBeTruncatingLog, + states.stateValue(tp).isTruncatingLog) + + assertEquals( + s"Partition $tp should${if (!shouldBeDelayed) " NOT" else ""} be delayed", + shouldBeDelayed, + states.stateValue(tp).isDelayed) + } + } + @Test def shouldHandleExceptionFromBlockingSend(): Unit = { val props = TestUtils.createBrokerConfig(1, "localhost:1234") @@ -108,8 +223,7 @@ class ReplicaFetcherThreadTest { val config = KafkaConfig.fromProps(TestUtils.createBrokerConfig(1, "localhost:1234")) //Setup all dependencies - val quota = createNiceMock(classOf[ReplicationQuotaManager]) - val leaderEpochs = createNiceMock(classOf[LeaderEpochCache]) + val leaderEpochs = createNiceMock(classOf[LeaderEpochFileCache]) val logManager = createMock(classOf[LogManager]) val replicaAlterLogDirsManager = createMock(classOf[ReplicaAlterLogDirsManager]) val replica = createNiceMock(classOf[Replica]) @@ -123,7 +237,7 @@ class ReplicaFetcherThreadTest { expect(replica.logEndOffset).andReturn(new LogOffsetMetadata(0)).anyTimes() expect(replica.highWatermark).andReturn(new LogOffsetMetadata(0)).anyTimes() expect(leaderEpochs.latestEpoch).andReturn(leaderEpoch) - expect(leaderEpochs.endOffsetFor(leaderEpoch)).andReturn((leaderEpoch, 0)).anyTimes() + expect(replica.endOffsetFor(leaderEpoch)).andReturn((leaderEpoch, 0)).anyTimes() expect(replicaManager.logManager).andReturn(logManager).anyTimes() expect(replicaManager.replicaAlterLogDirsManager).andReturn(replicaAlterLogDirsManager).anyTimes() stub(replica, partition, replicaManager) @@ -131,14 +245,15 @@ class ReplicaFetcherThreadTest { //Expectations expect(partition.truncateTo(anyLong(), anyBoolean())).once - replay(leaderEpochs, replicaManager, logManager, quota, replica) + replay(leaderEpochs, replicaManager, logManager, replica) //Define the offsets for the OffsetsForLeaderEpochResponse val offsets = Map(t1p0 -> new EpochEndOffset(leaderEpoch, 1), t1p1 -> new EpochEndOffset(leaderEpoch, 1)).asJava //Create the fetcher thread val mockNetwork = new ReplicaFetcherMockBlockingSend(offsets, brokerEndPoint, new SystemTime()) - val thread = new ReplicaFetcherThread("bob", 0, brokerEndPoint, config, replicaManager, new Metrics(), new SystemTime(), quota, Some(mockNetwork)) + val thread = new ReplicaFetcherThread("bob", 0, brokerEndPoint, config, replicaManager, + new Metrics, new SystemTime, UnboundedQuota, Some(mockNetwork)) thread.addPartitions(Map(t1p0 -> 0, t1p1 -> 0)) //Loop 1 @@ -169,7 +284,7 @@ class ReplicaFetcherThreadTest { // Setup all the dependencies val configs = TestUtils.createBrokerConfigs(1, "localhost:1234").map(KafkaConfig.fromProps) val quota = createNiceMock(classOf[ReplicationQuotaManager]) - val leaderEpochs = createMock(classOf[LeaderEpochCache]) + val leaderEpochs = createMock(classOf[LeaderEpochFileCache]) val logManager = createMock(classOf[LogManager]) val replicaAlterLogDirsManager = createMock(classOf[ReplicaAlterLogDirsManager]) val replica = createNiceMock(classOf[Replica]) @@ -185,7 +300,7 @@ class ReplicaFetcherThreadTest { expect(replica.logEndOffset).andReturn(new LogOffsetMetadata(initialLEO)).anyTimes() expect(replica.highWatermark).andReturn(new LogOffsetMetadata(initialLEO - 1)).anyTimes() expect(leaderEpochs.latestEpoch).andReturn(leaderEpoch).anyTimes() - expect(leaderEpochs.endOffsetFor(leaderEpoch)).andReturn((leaderEpoch, initialLEO)).anyTimes() + expect(replica.endOffsetFor(leaderEpoch)).andReturn((leaderEpoch, initialLEO)).anyTimes() expect(replicaManager.logManager).andReturn(logManager).anyTimes() expect(replicaManager.replicaAlterLogDirsManager).andReturn(replicaAlterLogDirsManager).anyTimes() stub(replica, partition, replicaManager) @@ -218,7 +333,7 @@ class ReplicaFetcherThreadTest { // Setup all the dependencies val configs = TestUtils.createBrokerConfigs(1, "localhost:1234").map(KafkaConfig.fromProps) val quota = createNiceMock(classOf[ReplicationQuotaManager]) - val leaderEpochs = createMock(classOf[LeaderEpochCache]) + val leaderEpochs = createMock(classOf[LeaderEpochFileCache]) val logManager = createMock(classOf[LogManager]) val replicaAlterLogDirsManager = createMock(classOf[ReplicaAlterLogDirsManager]) val replica = createNiceMock(classOf[Replica]) @@ -235,7 +350,7 @@ class ReplicaFetcherThreadTest { expect(replica.logEndOffset).andReturn(new LogOffsetMetadata(initialLEO)).anyTimes() expect(replica.highWatermark).andReturn(new LogOffsetMetadata(initialLEO - 3)).anyTimes() expect(leaderEpochs.latestEpoch).andReturn(leaderEpochAtFollower).anyTimes() - expect(leaderEpochs.endOffsetFor(leaderEpochAtLeader)).andReturn((UNDEFINED_EPOCH, UNDEFINED_EPOCH_OFFSET)).anyTimes() + expect(replica.endOffsetFor(leaderEpochAtLeader)).andReturn((UNDEFINED_EPOCH, UNDEFINED_EPOCH_OFFSET)).anyTimes() expect(replicaManager.logManager).andReturn(logManager).anyTimes() expect(replicaManager.replicaAlterLogDirsManager).andReturn(replicaAlterLogDirsManager).anyTimes() stub(replica, partition, replicaManager) @@ -272,7 +387,7 @@ class ReplicaFetcherThreadTest { // Setup all dependencies val quota = createNiceMock(classOf[ReplicationQuotaManager]) - val leaderEpochs = createNiceMock(classOf[LeaderEpochCache]) + val leaderEpochs = createNiceMock(classOf[LeaderEpochFileCache]) val logManager = createMock(classOf[LogManager]) val replicaAlterLogDirsManager = createMock(classOf[ReplicaAlterLogDirsManager]) val replica = createNiceMock(classOf[Replica]) @@ -287,8 +402,8 @@ class ReplicaFetcherThreadTest { expect(replica.logEndOffset).andReturn(new LogOffsetMetadata(initialLEO)).anyTimes() expect(replica.highWatermark).andReturn(new LogOffsetMetadata(initialLEO - 2)).anyTimes() expect(leaderEpochs.latestEpoch).andReturn(5) - expect(leaderEpochs.endOffsetFor(4)).andReturn((3, 120)).anyTimes() - expect(leaderEpochs.endOffsetFor(3)).andReturn((3, 120)).anyTimes() + expect(replica.endOffsetFor(4)).andReturn((3, 120)).anyTimes() + expect(replica.endOffsetFor(3)).andReturn((3, 120)).anyTimes() expect(replicaManager.logManager).andReturn(logManager).anyTimes() expect(replicaManager.replicaAlterLogDirsManager).andReturn(replicaAlterLogDirsManager).anyTimes() stub(replica, partition, replicaManager) @@ -343,7 +458,7 @@ class ReplicaFetcherThreadTest { // Setup all dependencies val quota = createNiceMock(classOf[ReplicationQuotaManager]) - val leaderEpochs = createNiceMock(classOf[LeaderEpochCache]) + val leaderEpochs = createNiceMock(classOf[LeaderEpochFileCache]) val logManager = createMock(classOf[LogManager]) val replicaAlterLogDirsManager = createMock(classOf[ReplicaAlterLogDirsManager]) val replica = createNiceMock(classOf[Replica]) @@ -404,7 +519,7 @@ class ReplicaFetcherThreadTest { // Setup all the dependencies val configs = TestUtils.createBrokerConfigs(1, "localhost:1234").map(KafkaConfig.fromProps) val quota = createNiceMock(classOf[ReplicationQuotaManager]) - val leaderEpochs = createNiceMock(classOf[LeaderEpochCache]) + val leaderEpochs = createNiceMock(classOf[LeaderEpochFileCache]) val logManager = createMock(classOf[LogManager]) val replicaAlterLogDirsManager = createMock(classOf[ReplicaAlterLogDirsManager]) val replica = createNiceMock(classOf[Replica]) @@ -449,7 +564,7 @@ class ReplicaFetcherThreadTest { // Setup all the dependencies val configs = TestUtils.createBrokerConfigs(1, "localhost:1234").map(KafkaConfig.fromProps) val quota = createNiceMock(classOf[kafka.server.ReplicationQuotaManager]) - val leaderEpochs = createNiceMock(classOf[LeaderEpochCache]) + val leaderEpochs = createNiceMock(classOf[LeaderEpochFileCache]) val logManager = createMock(classOf[kafka.log.LogManager]) val replicaAlterLogDirsManager = createMock(classOf[ReplicaAlterLogDirsManager]) val replica = createNiceMock(classOf[Replica]) @@ -467,7 +582,7 @@ class ReplicaFetcherThreadTest { expect(replica.logEndOffset).andReturn(new LogOffsetMetadata(initialLeo)).anyTimes() expect(leaderEpochs.latestEpoch).andReturn(leaderEpoch) // this is for the last reply with EpochEndOffset(5, 156) - expect(leaderEpochs.endOffsetFor(leaderEpoch)).andReturn((leaderEpoch, initialLeo)).anyTimes() + expect(replica.endOffsetFor(leaderEpoch)).andReturn((leaderEpoch, initialLeo)).anyTimes() expect(replicaManager.logManager).andReturn(logManager).anyTimes() expect(replicaManager.replicaAlterLogDirsManager).andReturn(replicaAlterLogDirsManager).anyTimes() stub(replica, partition, replicaManager) @@ -507,7 +622,7 @@ class ReplicaFetcherThreadTest { //Setup all stubs val quota = createNiceMock(classOf[ReplicationQuotaManager]) - val leaderEpochs = createNiceMock(classOf[LeaderEpochCache]) + val leaderEpochs = createNiceMock(classOf[LeaderEpochFileCache]) val logManager = createNiceMock(classOf[LogManager]) val replicaAlterLogDirsManager = createMock(classOf[ReplicaAlterLogDirsManager]) val replica = createNiceMock(classOf[Replica]) @@ -521,7 +636,7 @@ class ReplicaFetcherThreadTest { expect(replica.logEndOffset).andReturn(new LogOffsetMetadata(0)).anyTimes() expect(replica.highWatermark).andReturn(new LogOffsetMetadata(0)).anyTimes() expect(leaderEpochs.latestEpoch).andReturn(leaderEpoch) - expect(leaderEpochs.endOffsetFor(leaderEpoch)).andReturn((leaderEpoch, 0)).anyTimes() + expect(replica.endOffsetFor(leaderEpoch)).andReturn((leaderEpoch, 0)).anyTimes() expect(replicaManager.logManager).andReturn(logManager).anyTimes() expect(replicaManager.replicaAlterLogDirsManager).andReturn(replicaAlterLogDirsManager).anyTimes() stub(replica, partition, replicaManager) @@ -558,7 +673,7 @@ class ReplicaFetcherThreadTest { //Setup all stubs val quota = createNiceMock(classOf[ReplicationQuotaManager]) - val leaderEpochs = createNiceMock(classOf[LeaderEpochCache]) + val leaderEpochs = createNiceMock(classOf[LeaderEpochFileCache]) val logManager = createNiceMock(classOf[LogManager]) val replicaAlterLogDirsManager = createMock(classOf[ReplicaAlterLogDirsManager]) val replica = createNiceMock(classOf[Replica]) @@ -571,7 +686,7 @@ class ReplicaFetcherThreadTest { expect(replica.logEndOffset).andReturn(new LogOffsetMetadata(initialLEO)).anyTimes() expect(replica.highWatermark).andReturn(new LogOffsetMetadata(initialLEO - 2)).anyTimes() expect(leaderEpochs.latestEpoch).andReturn(5) - expect(leaderEpochs.endOffsetFor(5)).andReturn((5, initialLEO)).anyTimes() + expect(replica.endOffsetFor(5)).andReturn((5, initialLEO)).anyTimes() expect(replicaManager.logManager).andReturn(logManager).anyTimes() expect(replicaManager.replicaAlterLogDirsManager).andReturn(replicaAlterLogDirsManager).anyTimes() stub(replica, partition, replicaManager) @@ -603,6 +718,37 @@ class ReplicaFetcherThreadTest { assertEquals(49, truncateToCapture.getValue) } + @Test + def shouldCatchExceptionFromBlockingSendWhenShuttingDownReplicaFetcherThread(): Unit = { + val props = TestUtils.createBrokerConfig(1, "localhost:1234") + val config = KafkaConfig.fromProps(props) + val mockBlockingSend = createMock(classOf[BlockingSend]) + + expect(mockBlockingSend.initiateClose()).andThrow(new IllegalArgumentException()).once() + expect(mockBlockingSend.close()).andThrow(new IllegalStateException()).once() + replay(mockBlockingSend) + + val thread = new ReplicaFetcherThread( + name = "bob", + fetcherId = 0, + sourceBroker = brokerEndPoint, + brokerConfig = config, + replicaMgr = null, + metrics = new Metrics(), + time = new SystemTime(), + quota = null, + leaderEndpointBlockingSend = Some(mockBlockingSend)) + thread.start() + + // Verify that: + // 1) IllegalArgumentException thrown by BlockingSend#initiateClose() during `initiateShutdown` is not propagated + // 2) BlockingSend.close() is invoked even if BlockingSend#initiateClose() fails + // 3) IllegalStateException thrown by BlockingSend.close() during `awaitShutdown` is not propagated + thread.initiateShutdown() + thread.awaitShutdown() + verify(mockBlockingSend) + } + def stub(replica: Replica, partition: Partition, replicaManager: ReplicaManager) = { expect(replicaManager.getReplica(t1p0)).andReturn(Some(replica)).anyTimes() expect(replicaManager.getReplicaOrException(t1p0)).andReturn(replica).anyTimes() diff --git a/core/src/test/scala/unit/kafka/server/ReplicaManagerQuotasTest.scala b/core/src/test/scala/unit/kafka/server/ReplicaManagerQuotasTest.scala index c6efca573de7e..66a2c8e5fec11 100644 --- a/core/src/test/scala/unit/kafka/server/ReplicaManagerQuotasTest.scala +++ b/core/src/test/scala/unit/kafka/server/ReplicaManagerQuotasTest.scala @@ -52,8 +52,8 @@ class ReplicaManagerQuotasTest { val followerReplicaId = configs.last.brokerId val quota = mockQuota(1000000) - expect(quota.isQuotaExceeded()).andReturn(false).once() - expect(quota.isQuotaExceeded()).andReturn(true).once() + expect(quota.isQuotaExceeded).andReturn(false).once() + expect(quota.isQuotaExceeded).andReturn(true).once() replay(quota) val fetch = replicaManager.readFromLocalLog( @@ -78,8 +78,8 @@ class ReplicaManagerQuotasTest { val followerReplicaId = configs.last.brokerId val quota = mockQuota(1000000) - expect(quota.isQuotaExceeded()).andReturn(true).once() - expect(quota.isQuotaExceeded()).andReturn(true).once() + expect(quota.isQuotaExceeded).andReturn(true).once() + expect(quota.isQuotaExceeded).andReturn(true).once() replay(quota) val fetch = replicaManager.readFromLocalLog( @@ -103,8 +103,8 @@ class ReplicaManagerQuotasTest { val followerReplicaId = configs.last.brokerId val quota = mockQuota(1000000) - expect(quota.isQuotaExceeded()).andReturn(false).once() - expect(quota.isQuotaExceeded()).andReturn(false).once() + expect(quota.isQuotaExceeded).andReturn(false).once() + expect(quota.isQuotaExceeded).andReturn(false).once() replay(quota) val fetch = replicaManager.readFromLocalLog( @@ -128,8 +128,8 @@ class ReplicaManagerQuotasTest { val followerReplicaId = configs.last.brokerId val quota = mockQuota(1000000) - expect(quota.isQuotaExceeded()).andReturn(false).once() - expect(quota.isQuotaExceeded()).andReturn(true).once() + expect(quota.isQuotaExceeded).andReturn(false).once() + expect(quota.isQuotaExceeded).andReturn(true).once() replay(quota) val fetch = replicaManager.readFromLocalLog( diff --git a/core/src/test/scala/unit/kafka/server/ReplicaManagerTest.scala b/core/src/test/scala/unit/kafka/server/ReplicaManagerTest.scala index ce8868861f78b..84841519f7fa1 100644 --- a/core/src/test/scala/unit/kafka/server/ReplicaManagerTest.scala +++ b/core/src/test/scala/unit/kafka/server/ReplicaManagerTest.scala @@ -19,21 +19,26 @@ package kafka.server import java.io.File import java.util.Properties +import java.util.concurrent.{CountDownLatch, TimeUnit} import java.util.concurrent.atomic.AtomicBoolean -import kafka.log.LogConfig +import kafka.log.{Log, LogConfig, LogManager, ProducerStateManager} import kafka.utils.{MockScheduler, MockTime, TestUtils} import TestUtils.createBroker +import kafka.cluster.BrokerEndPoint +import kafka.server.epoch.LeaderEpochFileCache +import kafka.server.epoch.util.ReplicaFetcherMockBlockingSend import kafka.utils.timer.MockTimer import kafka.zk.KafkaZkClient import org.I0Itec.zkclient.ZkClient import org.apache.kafka.common.metrics.Metrics import org.apache.kafka.common.protocol.{ApiKeys, Errors} import org.apache.kafka.common.record._ -import org.apache.kafka.common.requests.{IsolationLevel, LeaderAndIsrRequest} +import org.apache.kafka.common.requests.{EpochEndOffset, IsolationLevel, LeaderAndIsrRequest} import org.apache.kafka.common.requests.ProduceResponse.PartitionResponse import org.apache.kafka.common.requests.FetchRequest.PartitionData import org.apache.kafka.common.requests.FetchResponse.AbortedTransaction +import org.apache.kafka.common.utils.Time import org.apache.kafka.common.{Node, TopicPartition} import org.apache.zookeeper.data.Stat import org.easymock.EasyMock @@ -51,6 +56,11 @@ class ReplicaManagerTest { var zkClient: ZkClient = _ var kafkaZkClient: KafkaZkClient = _ + // Constants defined for readability + val zkVersion = 0 + val correlationId = 0 + var controllerEpoch = 0 + @Before def setUp() { zkClient = EasyMock.createMock(classOf[ZkClient]) @@ -463,7 +473,9 @@ class ReplicaManagerTest { val tp0Status = responseStatusMap.get(tp0) assertTrue(tp0Status.isDefined) - assertEquals(1, tp0Status.get.highWatermark) + // the response contains high watermark on the leader before it is updated based + // on this fetch request + assertEquals(0, tp0Status.get.highWatermark) assertEquals(None, tp0Status.get.lastStableOffset) assertEquals(Errors.NONE, tp0Status.get.error) assertTrue(tp0Status.get.records.batches.iterator.hasNext) @@ -502,6 +514,216 @@ class ReplicaManagerTest { } } + /** + * If a partition becomes a follower and the leader is unchanged it should check for truncation + * if the epoch has increased by more than one (which suggests it has missed an update) + */ + @Test + def testBecomeFollowerWhenLeaderIsUnchangedButMissedLeaderUpdate() { + val topicPartition = 0 + val followerBrokerId = 0 + val leaderBrokerId = 1 + val controllerId = 0 + val controllerEpoch = 0 + var leaderEpoch = 1 + val aliveBrokerIds = Seq[Integer] (followerBrokerId, leaderBrokerId) + val countDownLatch = new CountDownLatch(1) + + // Prepare the mocked components for the test + val (replicaManager, mockLogMgr) = prepareReplicaManagerAndLogManager( + topicPartition, followerBrokerId, leaderBrokerId, countDownLatch, expectTruncation = true) + + // Initialize partition state to follower, with leader = 1, leaderEpoch = 1 + val partition = replicaManager.getOrCreatePartition(new TopicPartition(topic, topicPartition)) + partition.getOrCreateReplica(followerBrokerId) + partition.makeFollower(controllerId, + leaderAndIsrPartitionState(leaderEpoch, leaderBrokerId, aliveBrokerIds), + correlationId) + + // Make local partition a follower - because epoch increased by more than 1, truncation should + // trigger even though leader does not change + leaderEpoch += 2 + val leaderAndIsrRequest0 = new LeaderAndIsrRequest.Builder(ApiKeys.LEADER_AND_ISR.latestVersion, + controllerId, controllerEpoch, + collection.immutable.Map(new TopicPartition(topic, topicPartition) -> + leaderAndIsrPartitionState(leaderEpoch, leaderBrokerId, aliveBrokerIds)).asJava, + Set(new Node(followerBrokerId, "host1", 0), + new Node(leaderBrokerId, "host2", 1)).asJava).build() + replicaManager.becomeLeaderOrFollower(correlationId, leaderAndIsrRequest0, + (_, followers) => assertEquals(followerBrokerId, followers.head.partitionId)) + assertTrue(countDownLatch.await(1000L, TimeUnit.MILLISECONDS)) + + // Truncation should have happened once + EasyMock.verify(mockLogMgr) + } + + /** + * If a partition becomes a follower and the leader is unchanged but no epoch update + * has been missed, it should not check for truncation + */ + @Test + def testDontBecomeFollowerWhenNoMissedLeaderUpdate() { + val topicPartition = 0 + val followerBrokerId = 0 + val leaderBrokerId = 1 + val controllerId = 0 + var leaderEpoch = 1 + val aliveBrokerIds = Seq[Integer] (followerBrokerId, leaderBrokerId) + val countDownLatch = new CountDownLatch(1) + + // Prepare the mocked components for the test + val (replicaManager, mockLogMgr) = prepareReplicaManagerAndLogManager( + topicPartition, followerBrokerId, leaderBrokerId, countDownLatch, expectTruncation = false) + + // Initialize partition state to follower, with leader = 1, leaderEpoch = 1 + val partition = replicaManager.getOrCreatePartition(new TopicPartition(topic, topicPartition)) + partition.getOrCreateReplica(followerBrokerId) + partition.makeFollower(controllerId, + leaderAndIsrPartitionState(leaderEpoch, leaderBrokerId, aliveBrokerIds), + correlationId) + + // Make local partition a follower - because epoch did not change, truncation should not trigger + val leaderAndIsrRequest0 = new LeaderAndIsrRequest.Builder(ApiKeys.LEADER_AND_ISR.latestVersion, + controllerId, controllerEpoch, + collection.immutable.Map(new TopicPartition(topic, topicPartition) -> + leaderAndIsrPartitionState(leaderEpoch, leaderBrokerId, aliveBrokerIds)).asJava, + Set(new Node(followerBrokerId, "host1", 0), + new Node(leaderBrokerId, "host2", 1)).asJava).build() + replicaManager.becomeLeaderOrFollower(correlationId, leaderAndIsrRequest0, + (_, followers) => assertTrue(followers.isEmpty)) + + // Make local partition a follower - because epoch increased by only 1 and leader did not change, + // truncation should not trigger + leaderEpoch += 1 + val leaderAndIsrRequest1 = new LeaderAndIsrRequest.Builder(ApiKeys.LEADER_AND_ISR.latestVersion, + controllerId, controllerEpoch, + collection.immutable.Map(new TopicPartition(topic, topicPartition) -> + leaderAndIsrPartitionState(leaderEpoch, leaderBrokerId, aliveBrokerIds)).asJava, + Set(new Node(followerBrokerId, "host1", 0), + new Node(leaderBrokerId, "host2", 1)).asJava).build() + replicaManager.becomeLeaderOrFollower(correlationId, leaderAndIsrRequest1, + (_, followers) => assertTrue(followers.isEmpty)) + + // Truncation should not have happened + EasyMock.verify(mockLogMgr) + } + + private def prepareReplicaManagerAndLogManager(topicPartition: Int, + followerBrokerId: Int, + leaderBrokerId: Int, + countDownLatch: CountDownLatch, + expectTruncation: Boolean) : (ReplicaManager, LogManager) = { + val props = TestUtils.createBrokerConfig(0, TestUtils.MockZkConnect) + props.put("log.dir", TestUtils.tempRelativeDir("data").getAbsolutePath) + val config = KafkaConfig.fromProps(props) + + // Setup mock local log to have leader epoch of 3 and offset of 10 + val localLogOffset = 10 + val offsetFromLeader = 5 + val leaderEpochFromLeader = 3 + val mockScheduler = new MockScheduler(time) + val mockBrokerTopicStats = new BrokerTopicStats + val mockLogDirFailureChannel = new LogDirFailureChannel(config.logDirs.size) + val mockLeaderEpochCache = EasyMock.createMock(classOf[LeaderEpochFileCache]) + EasyMock.expect(mockLeaderEpochCache.latestEpoch).andReturn(leaderEpochFromLeader).anyTimes() + EasyMock.expect(mockLeaderEpochCache.endOffsetFor(leaderEpochFromLeader)) + .andReturn((leaderEpochFromLeader, localLogOffset)) + EasyMock.replay(mockLeaderEpochCache) + val mockLog = new Log( + dir = new File(new File(config.logDirs.head), s"$topic-0"), + config = LogConfig(), + logStartOffset = 0L, + recoveryPoint = 0L, + scheduler = mockScheduler, + brokerTopicStats = mockBrokerTopicStats, + time = time, + maxProducerIdExpirationMs = 30000, + producerIdExpirationCheckIntervalMs = 30000, + topicPartition = new TopicPartition(topic, topicPartition), + producerStateManager = new ProducerStateManager(new TopicPartition(topic, topicPartition), + new File(new File(config.logDirs.head), s"$topic-$topicPartition"), 30000), + logDirFailureChannel = mockLogDirFailureChannel) { + + override def leaderEpochCache: LeaderEpochFileCache = mockLeaderEpochCache + + override def logEndOffsetMetadata = LogOffsetMetadata(localLogOffset) + } + + // Expect to call LogManager.truncateTo exactly once + val mockLogMgr = EasyMock.createMock(classOf[LogManager]) + EasyMock.expect(mockLogMgr.liveLogDirs).andReturn(config.logDirs.map(new File(_).getAbsoluteFile)).anyTimes + EasyMock.expect(mockLogMgr.currentDefaultConfig).andReturn(LogConfig()) + EasyMock.expect(mockLogMgr.getOrCreateLog(new TopicPartition(topic, topicPartition), + LogConfig(), isNew = false, isFuture = false)).andReturn(mockLog).anyTimes + if (expectTruncation) { + EasyMock.expect(mockLogMgr.truncateTo(Map(new TopicPartition(topic, topicPartition) -> offsetFromLeader), + isFuture = false)).once + } + EasyMock.replay(mockLogMgr) + + val aliveBrokerIds = Seq[Integer](followerBrokerId, leaderBrokerId) + val aliveBrokers = aliveBrokerIds.map(brokerId => createBroker(brokerId, s"host$brokerId", brokerId)) + + val metadataCache = EasyMock.createMock(classOf[MetadataCache]) + EasyMock.expect(metadataCache.getAliveBrokers).andReturn(aliveBrokers).anyTimes + aliveBrokerIds.foreach { brokerId => + EasyMock.expect(metadataCache.isBrokerAlive(EasyMock.eq(brokerId))).andReturn(true).anyTimes + } + EasyMock.replay(metadataCache) + + val timer = new MockTimer + val mockProducePurgatory = new DelayedOperationPurgatory[DelayedProduce]( + purgatoryName = "Produce", timer, reaperEnabled = false) + val mockFetchPurgatory = new DelayedOperationPurgatory[DelayedFetch]( + purgatoryName = "Fetch", timer, reaperEnabled = false) + val mockDeleteRecordsPurgatory = new DelayedOperationPurgatory[DelayedDeleteRecords]( + purgatoryName = "DeleteRecords", timer, reaperEnabled = false) + + // Mock network client to show leader offset of 5 + val quota = QuotaFactory.instantiate(config, metrics, time, "") + val blockingSend = new ReplicaFetcherMockBlockingSend(Map(new TopicPartition(topic, topicPartition) -> + new EpochEndOffset(leaderEpochFromLeader, offsetFromLeader)).asJava, BrokerEndPoint(1, "host1" ,1), time) + val replicaManager = new ReplicaManager(config, metrics, time, kafkaZkClient, mockScheduler, mockLogMgr, + new AtomicBoolean(false), quota, mockBrokerTopicStats, + metadataCache, mockLogDirFailureChannel, mockProducePurgatory, mockFetchPurgatory, + mockDeleteRecordsPurgatory, Option(this.getClass.getName)) { + + override protected def createReplicaFetcherManager(metrics: Metrics, + time: Time, + threadNamePrefix: Option[String], + quotaManager: ReplicationQuotaManager): ReplicaFetcherManager = { + new ReplicaFetcherManager(config, this, metrics, time, threadNamePrefix, quotaManager) { + + override def createFetcherThread(fetcherId: Int, sourceBroker: BrokerEndPoint): AbstractFetcherThread = { + new ReplicaFetcherThread(s"ReplicaFetcherThread-$fetcherId", fetcherId, + sourceBroker, config, replicaManager, metrics, time, quota.follower, Some(blockingSend)) { + + override def doWork() = { + // In case the thread starts before the partition is added by AbstractFetcherManager, + // add it here (it's a no-op if already added) + addPartitions(Map(new TopicPartition(topic, topicPartition) -> 0L)) + super.doWork() + + // Shut the thread down after one iteration to avoid double-counting truncations + initiateShutdown() + countDownLatch.countDown() + } + } + } + } + } + } + + (replicaManager, mockLogMgr) + } + + private def leaderAndIsrPartitionState(leaderEpoch: Int, + leaderBrokerId: Int, + aliveBrokerIds: Seq[Integer]) : LeaderAndIsrRequest.PartitionState = { + new LeaderAndIsrRequest.PartitionState(controllerEpoch, leaderBrokerId, leaderEpoch, aliveBrokerIds.asJava, + zkVersion, aliveBrokerIds.asJava, false) + } + private class CallbackResult[T] { private var value: Option[T] = None private var fun: Option[T => Unit] = None @@ -530,7 +752,8 @@ class ReplicaManagerTest { private def appendRecords(replicaManager: ReplicaManager, partition: TopicPartition, records: MemoryRecords, - isFromClient: Boolean = true): CallbackResult[PartitionResponse] = { + isFromClient: Boolean = true, + requiredAcks: Short = -1): CallbackResult[PartitionResponse] = { val result = new CallbackResult[PartitionResponse]() def appendCallback(responses: Map[TopicPartition, PartitionResponse]): Unit = { val response = responses.get(partition) @@ -540,7 +763,7 @@ class ReplicaManagerTest { replicaManager.appendRecords( timeout = 1000, - requiredAcks = -1, + requiredAcks = requiredAcks, internalTopicsAllowed = false, isFromClient = isFromClient, entriesPerPartition = Map(partition -> records), diff --git a/core/src/test/scala/unit/kafka/server/ReplicationQuotaManagerTest.scala b/core/src/test/scala/unit/kafka/server/ReplicationQuotaManagerTest.scala index 54b506d768a6e..b1edd010020d2 100644 --- a/core/src/test/scala/unit/kafka/server/ReplicationQuotaManagerTest.scala +++ b/core/src/test/scala/unit/kafka/server/ReplicationQuotaManagerTest.scala @@ -50,7 +50,7 @@ class ReplicationQuotaManagerTest { quota.updateQuota(new Quota(100, true)) //Quota should not be broken when we start - assertFalse(quota.isQuotaExceeded()) + assertFalse(quota.isQuotaExceeded) //First window is fixed, so we'll skip it time.sleep(1000) @@ -60,24 +60,24 @@ class ReplicationQuotaManagerTest { quota.record(1) //Then it should not break the quota - assertFalse(quota.isQuotaExceeded()) + assertFalse(quota.isQuotaExceeded) //When we record half the quota (half way through the window), we still should not break quota.record(149) //150B, 1.5s - assertFalse(quota.isQuotaExceeded()) + assertFalse(quota.isQuotaExceeded) //Add a byte to push over quota quota.record(1) //151B, 1.5s //Then it should break the quota assertEquals(151 / 1.5, rate(metrics), 0) //151B, 1.5s - assertTrue(quota.isQuotaExceeded()) + assertTrue(quota.isQuotaExceeded) //When we sleep for the remaining half the window time.sleep(500) //151B, 2s //Then Our rate should have halved (i.e back down below the quota) - assertFalse(quota.isQuotaExceeded()) + assertFalse(quota.isQuotaExceeded) assertEquals(151d / 2, rate(metrics), 0.1) //151B, 2s //When we sleep for another half a window (now half way through second window) @@ -86,14 +86,14 @@ class ReplicationQuotaManagerTest { //Then the rate should be exceeded again assertEquals(250 / 2.5, rate(metrics), 0) //250B, 2.5s - assertFalse(quota.isQuotaExceeded()) + assertFalse(quota.isQuotaExceeded) quota.record(1) - assertTrue(quota.isQuotaExceeded()) + assertTrue(quota.isQuotaExceeded) assertEquals(251 / 2.5, rate(metrics), 0) //Sleep for 2 more window time.sleep(2 * 1000) //so now at 3.5s - assertFalse(quota.isQuotaExceeded()) + assertFalse(quota.isQuotaExceeded) assertEquals(251d / 4.5, rate(metrics), 0) } diff --git a/core/src/test/scala/unit/kafka/server/ReplicationQuotasTest.scala b/core/src/test/scala/unit/kafka/server/ReplicationQuotasTest.scala index 45a6bddc7d11f..3fac95c9a288e 100644 --- a/core/src/test/scala/unit/kafka/server/ReplicationQuotasTest.scala +++ b/core/src/test/scala/unit/kafka/server/ReplicationQuotasTest.scala @@ -111,7 +111,7 @@ class ReplicationQuotasTest extends ZooKeeperTestHarness { adminZkClient.changeTopicConfig(topic, propsWith(FollowerReplicationThrottledReplicasProp, "0:106,1:106,2:106,3:107,4:107,5:107")) //Add data equally to each partition - producer = createNewProducer(getBrokerListStrFromServers(brokers), retries = 5, acks = 1) + producer = createProducer(getBrokerListStrFromServers(brokers), retries = 5, acks = 1) (0 until msgCount).foreach { _ => (0 to 7).foreach { partition => producer.send(new ProducerRecord(topic, partition, null, msg)) @@ -207,7 +207,7 @@ class ReplicationQuotasTest extends ZooKeeperTestHarness { } def addData(msgCount: Int, msg: Array[Byte]): Unit = { - producer = createNewProducer(getBrokerListStrFromServers(brokers), retries = 5, acks = 0) + producer = createProducer(getBrokerListStrFromServers(brokers), retries = 5, acks = 0) (0 until msgCount).map(_ => producer.send(new ProducerRecord(topic, msg))).foreach(_.get) waitForOffsetsToMatch(msgCount, 0, 100) } diff --git a/core/src/test/scala/unit/kafka/server/RequestQuotaTest.scala b/core/src/test/scala/unit/kafka/server/RequestQuotaTest.scala index 59f543b522a1f..0205bcf214156 100644 --- a/core/src/test/scala/unit/kafka/server/RequestQuotaTest.scala +++ b/core/src/test/scala/unit/kafka/server/RequestQuotaTest.scala @@ -24,7 +24,8 @@ import kafka.security.auth._ import kafka.utils.TestUtils import org.apache.kafka.clients.admin.NewPartitions import org.apache.kafka.common.acl.{AccessControlEntry, AccessControlEntryFilter, AclBinding, AclBindingFilter, AclOperation, AclPermissionType} -import org.apache.kafka.common.resource.{ResourceFilter, ResourceNameType, Resource => AdminResource, ResourceType => AdminResourceType} +import org.apache.kafka.common.config.ConfigResource +import org.apache.kafka.common.resource.{PatternType, ResourcePattern, ResourcePatternFilter, ResourceType => AdminResourceType} import org.apache.kafka.common.{Node, TopicPartition} import org.apache.kafka.common.metrics.{KafkaMetric, Quota, Sensor} import org.apache.kafka.common.network.ListenerName @@ -32,7 +33,7 @@ import org.apache.kafka.common.protocol.ApiKeys import org.apache.kafka.common.protocol.types.Struct import org.apache.kafka.common.record._ import org.apache.kafka.common.requests.CreateAclsRequest.AclCreation -import org.apache.kafka.common.requests.{Resource => RResource, ResourceType => RResourceType, _} +import org.apache.kafka.common.requests._ import org.apache.kafka.common.security.auth.{AuthenticationContext, KafkaPrincipal, KafkaPrincipalBuilder, SecurityProtocol} import org.apache.kafka.common.utils.Sanitizer import org.apache.kafka.common.utils.SecurityUtils @@ -312,20 +313,20 @@ class RequestQuotaTest extends BaseRequestTest { case ApiKeys.CREATE_ACLS => new CreateAclsRequest.Builder(Collections.singletonList(new AclCreation(new AclBinding( - new AdminResource(AdminResourceType.TOPIC, "mytopic"), + new ResourcePattern(AdminResourceType.TOPIC, "mytopic", PatternType.LITERAL), new AccessControlEntry("User:ANONYMOUS", "*", AclOperation.WRITE, AclPermissionType.DENY))))) case ApiKeys.DELETE_ACLS => new DeleteAclsRequest.Builder(Collections.singletonList(new AclBindingFilter( - new ResourceFilter(AdminResourceType.TOPIC, null, ResourceNameType.LITERAL), + new ResourcePatternFilter(AdminResourceType.TOPIC, null, PatternType.LITERAL), new AccessControlEntryFilter("User:ANONYMOUS", "*", AclOperation.ANY, AclPermissionType.DENY)))) case ApiKeys.DESCRIBE_CONFIGS => - new DescribeConfigsRequest.Builder(Collections.singleton(new RResource(RResourceType.TOPIC, tp.topic))) + new DescribeConfigsRequest.Builder(Collections.singleton(new ConfigResource(ConfigResource.Type.TOPIC, tp.topic))) case ApiKeys.ALTER_CONFIGS => new AlterConfigsRequest.Builder( - Collections.singletonMap(new RResource(RResourceType.TOPIC, tp.topic), + Collections.singletonMap(new ConfigResource(ConfigResource.Type.TOPIC, tp.topic), new AlterConfigsRequest.Config(Collections.singleton( new AlterConfigsRequest.ConfigEntry(LogConfig.MaxMessageBytesProp, "1000000") ))), true) diff --git a/core/src/test/scala/unit/kafka/server/ServerShutdownTest.scala b/core/src/test/scala/unit/kafka/server/ServerShutdownTest.scala index 596c3539f8360..94cf6950f71be 100755 --- a/core/src/test/scala/unit/kafka/server/ServerShutdownTest.scala +++ b/core/src/test/scala/unit/kafka/server/ServerShutdownTest.scala @@ -17,20 +17,30 @@ package kafka.server import kafka.zk.ZooKeeperTestHarness -import kafka.consumer.SimpleConsumer import kafka.utils.{CoreUtils, TestUtils} import kafka.utils.TestUtils._ -import kafka.api.FetchRequestBuilder -import kafka.message.ByteBufferMessageSet -import java.io.File +import java.io.{DataInputStream, File} +import java.net.ServerSocket +import java.util.concurrent.{Executors, TimeUnit} +import kafka.cluster.Broker +import kafka.controller.{ControllerChannelManager, ControllerContext, StateChangeLogger} import kafka.log.LogManager +import kafka.zookeeper.ZooKeeperClientTimeoutException +import org.apache.kafka.clients.consumer.KafkaConsumer import org.apache.kafka.clients.producer.{KafkaProducer, ProducerRecord} import org.apache.kafka.common.errors.KafkaStorageException -import org.apache.kafka.common.serialization.{IntegerSerializer, StringSerializer} +import org.apache.kafka.common.metrics.Metrics +import org.apache.kafka.common.network.ListenerName +import org.apache.kafka.common.protocol.ApiKeys +import org.apache.kafka.common.requests.LeaderAndIsrRequest +import org.apache.kafka.common.security.auth.SecurityProtocol +import org.apache.kafka.common.serialization.{IntegerDeserializer, IntegerSerializer, StringDeserializer, StringSerializer} +import org.apache.kafka.common.utils.Time import org.junit.{Before, Test} import org.junit.Assert._ +import scala.collection.JavaConverters._ import scala.reflect.ClassTag class ServerShutdownTest extends ZooKeeperTestHarness { @@ -51,13 +61,21 @@ class ServerShutdownTest extends ZooKeeperTestHarness { def testCleanShutdown() { def createProducer(server: KafkaServer): KafkaProducer[Integer, String] = - TestUtils.createNewProducer( + TestUtils.createProducer( TestUtils.getBrokerListStrFromServers(Seq(server)), retries = 5, keySerializer = new IntegerSerializer, valueSerializer = new StringSerializer ) + def createConsumer(server: KafkaServer): KafkaConsumer[Integer, String] = + TestUtils.createConsumer( + TestUtils.getBrokerListStrFromServers(Seq(server)), + securityProtocol = SecurityProtocol.PLAINTEXT, + keyDeserializer = new IntegerDeserializer, + valueDeserializer = new StringDeserializer + ) + var server = new KafkaServer(config, threadNamePrefix = Option(this.getClass.getName)) server.startup() var producer = createProducer(server) @@ -85,25 +103,17 @@ class ServerShutdownTest extends ZooKeeperTestHarness { TestUtils.waitUntilMetadataIsPropagated(Seq(server), topic, 0) producer = createProducer(server) - val consumer = new SimpleConsumer(host, TestUtils.boundPort(server), 1000000, 64*1024, "") + val consumer = createConsumer(server) + consumer.subscribe(Seq(topic).asJava) - 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 => TestUtils.readString(m.message.payload))) - val newOffset = fetchedMessage.last.nextOffset + val consumerRecords = TestUtils.consumeRecords(consumer, sent1.size) + assertEquals(sent1, consumerRecords.map(_.value)) // send some more messages sent2.map(value => producer.send(new ProducerRecord(topic, 0, value))).foreach(_.get) - fetchedMessage = null - while (fetchedMessage == null || fetchedMessage.validBytes == 0) { - val fetched = consumer.fetch(new FetchRequestBuilder().addFetch(topic, 0, newOffset, 10000).build()) - fetchedMessage = fetched.messageSet(topic, 0) - } - assertEquals(sent2, fetchedMessage.map(m => TestUtils.readString(m.message.payload))) + val consumerRecords2 = TestUtils.consumeRecords(consumer, sent2.size) + assertEquals(sent2, consumerRecords2.map(_.value)) consumer.close() producer.close() @@ -128,9 +138,10 @@ class ServerShutdownTest extends ZooKeeperTestHarness { @Test def testCleanShutdownAfterFailedStartup() { val newProps = TestUtils.createBrokerConfig(0, zkConnect) - newProps.setProperty("zookeeper.connect", "some.invalid.hostname.foo.bar.local:65535") + newProps.setProperty(KafkaConfig.ZkConnectionTimeoutMsProp, "50") + newProps.setProperty(KafkaConfig.ZkConnectProp, "some.invalid.hostname.foo.bar.local:65535") val newConfig = KafkaConfig.fromProps(newProps) - verifyCleanShutdownAfterFailedStartup[IllegalArgumentException](newConfig) + verifyCleanShutdownAfterFailedStartup[ZooKeeperClientTimeoutException](newConfig) } @Test @@ -175,23 +186,71 @@ class ServerShutdownTest extends ZooKeeperTestHarness { } def verifyNonDaemonThreadsStatus() { - assertEquals(0, Thread.getAllStackTraces.keySet().toArray - .map{ _.asInstanceOf[Thread] } + assertEquals(0, Thread.getAllStackTraces.keySet.toArray + .map(_.asInstanceOf[Thread]) .count(isNonDaemonKafkaThread)) } @Test def testConsecutiveShutdown(){ val server = new KafkaServer(config) + server.startup() + server.shutdown() + server.awaitShutdown() + server.shutdown() + } + + // Verify that if controller is in the midst of processing a request, shutdown completes + // without waiting for request timeout. + @Test + def testControllerShutdownDuringSend(): Unit = { + val securityProtocol = SecurityProtocol.PLAINTEXT + val listenerName = ListenerName.forSecurityProtocol(securityProtocol) + + val controllerId = 2 + val metrics = new Metrics + val executor = Executors.newSingleThreadExecutor + var serverSocket: ServerSocket = null + var controllerChannelManager: ControllerChannelManager = null + try { - server.startup() - server.shutdown() - server.awaitShutdown() - server.shutdown() - assertTrue(true) - } - catch{ - case _: Throwable => fail() + // Set up a server to accept a connection and receive one byte from the first request. No response is sent. + serverSocket = new ServerSocket(0) + val receiveFuture = executor.submit(new Runnable { + override def run(): Unit = { + val socket = serverSocket.accept() + new DataInputStream(socket.getInputStream).readByte() + } + }) + + // Start a ControllerChannelManager + val brokers = Seq(new Broker(1, "localhost", serverSocket.getLocalPort, listenerName, securityProtocol)) + val controllerConfig = KafkaConfig.fromProps(TestUtils.createBrokerConfig(controllerId, zkConnect)) + val controllerContext = new ControllerContext + controllerContext.liveBrokers = brokers.toSet + controllerChannelManager = new ControllerChannelManager(controllerContext, controllerConfig, Time.SYSTEM, + metrics, new StateChangeLogger(controllerId, inControllerContext = true, None)) + controllerChannelManager.startup() + + // Initiate a sendRequest and wait until connection is established and one byte is received by the peer + val requestBuilder = new LeaderAndIsrRequest.Builder(ApiKeys.LEADER_AND_ISR.latestVersion, + controllerId, 1, Map.empty.asJava, brokers.map(_.node(listenerName)).toSet.asJava) + controllerChannelManager.sendRequest(1, ApiKeys.LEADER_AND_ISR, requestBuilder) + receiveFuture.get(10, TimeUnit.SECONDS) + + // Shutdown controller. Request timeout is 30s, verify that shutdown completed well before that + val shutdownFuture = executor.submit(new Runnable { + override def run(): Unit = controllerChannelManager.shutdown() + }) + shutdownFuture.get(10, TimeUnit.SECONDS) + + } finally { + if (serverSocket != null) + serverSocket.close() + if (controllerChannelManager != null) + controllerChannelManager.shutdown() + executor.shutdownNow() + metrics.close() } } } diff --git a/core/src/test/scala/unit/kafka/server/ServerStartupTest.scala b/core/src/test/scala/unit/kafka/server/ServerStartupTest.scala index 64647dee1e0b7..67d083c7ab966 100755 --- a/core/src/test/scala/unit/kafka/server/ServerStartupTest.scala +++ b/core/src/test/scala/unit/kafka/server/ServerStartupTest.scala @@ -17,9 +17,9 @@ package kafka.server -import kafka.common.KafkaException import kafka.utils.TestUtils import kafka.zk.ZooKeeperTestHarness +import org.apache.kafka.common.KafkaException import org.apache.zookeeper.KeeperException.NodeExistsException import org.easymock.EasyMock import org.junit.Assert._ diff --git a/core/src/test/scala/unit/kafka/server/StopReplicaRequestTest.scala b/core/src/test/scala/unit/kafka/server/StopReplicaRequestTest.scala new file mode 100644 index 0000000000000..5df61ebe56e03 --- /dev/null +++ b/core/src/test/scala/unit/kafka/server/StopReplicaRequestTest.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.server + +import kafka.utils._ +import org.apache.kafka.common.TopicPartition +import org.apache.kafka.common.protocol.{ApiKeys, Errors} +import org.apache.kafka.common.requests._ +import org.junit.Assert._ +import org.junit.Test +import collection.JavaConverters._ + + +class StopReplicaRequestTest extends BaseRequestTest { + override val logDirCount = 2 + override val numBrokers: Int = 1 + + val topic = "topic" + val partitionNum = 2 + val tp0 = new TopicPartition(topic, 0) + val tp1 = new TopicPartition(topic, 1) + + @Test + def testStopReplicaRequest(): Unit = { + createTopic(topic, partitionNum, 1) + TestUtils.generateAndProduceMessages(servers, topic, 10) + + val server = servers.head + val offlineDir = server.logManager.getLog(tp1).get.dir.getParent + server.replicaManager.handleLogDirFailure(offlineDir, sendZkNotification = false) + + for (i <- 1 to 2) { + val request1 = new StopReplicaRequest.Builder( + server.config.brokerId, server.replicaManager.controllerEpoch, true, Set(tp0, tp1).asJava).build() + val response1 = connectAndSend(request1, ApiKeys.STOP_REPLICA, controllerSocketServer) + val partitionErrors1 = StopReplicaResponse.parse(response1, request1.version).responses() + assertEquals(Errors.NONE, partitionErrors1.get(tp0)) + assertEquals(Errors.KAFKA_STORAGE_ERROR, partitionErrors1.get(tp1)) + } + } + +} diff --git a/core/src/test/scala/unit/kafka/server/checkpoints/LeaderEpochCheckpointFileTest.scala b/core/src/test/scala/unit/kafka/server/checkpoints/LeaderEpochCheckpointFileTest.scala index e7c6a9785bce6..0c47f15a09ff2 100644 --- a/core/src/test/scala/unit/kafka/server/checkpoints/LeaderEpochCheckpointFileTest.scala +++ b/core/src/test/scala/unit/kafka/server/checkpoints/LeaderEpochCheckpointFileTest.scala @@ -24,7 +24,6 @@ import org.junit.Assert._ import org.junit.Test import org.scalatest.junit.JUnitSuite - class LeaderEpochCheckpointFileTest extends JUnitSuite with Logging{ @Test diff --git a/core/src/test/scala/unit/kafka/server/epoch/EpochDrivenReplicationProtocolAcceptanceTest.scala b/core/src/test/scala/unit/kafka/server/epoch/EpochDrivenReplicationProtocolAcceptanceTest.scala index 05a6bb3255306..6facaa551aa05 100644 --- a/core/src/test/scala/unit/kafka/server/epoch/EpochDrivenReplicationProtocolAcceptanceTest.scala +++ b/core/src/test/scala/unit/kafka/server/epoch/EpochDrivenReplicationProtocolAcceptanceTest.scala @@ -79,7 +79,7 @@ class EpochDrivenReplicationProtocolAcceptanceTest extends ZooKeeperTestHarness //A single partition topic with 2 replicas adminZkClient.createOrUpdateTopicPartitionAssignmentPathInZK(topic, Map(0 -> Seq(100, 101))) - producer = createProducer() + producer = createProducer val tp = new TopicPartition(topic, 0) //When one record is written to the leader @@ -90,23 +90,23 @@ class EpochDrivenReplicationProtocolAcceptanceTest extends ZooKeeperTestHarness assertEquals(0, latestRecord(follower).partitionLeaderEpoch()) //Both leader and follower should have recorded Epoch 0 at Offset 0 - assertEquals(Buffer(EpochEntry(0, 0)), epochCache(leader).epochEntries()) - assertEquals(Buffer(EpochEntry(0, 0)), epochCache(follower).epochEntries()) + assertEquals(Buffer(EpochEntry(0, 0)), epochCache(leader).epochEntries) + assertEquals(Buffer(EpochEntry(0, 0)), epochCache(follower).epochEntries) //Bounce the follower bounce(follower) awaitISR(tp) //Nothing happens yet as we haven't sent any new messages. - assertEquals(Buffer(EpochEntry(0, 0)), epochCache(leader).epochEntries()) - assertEquals(Buffer(EpochEntry(0, 0)), epochCache(follower).epochEntries()) + assertEquals(Buffer(EpochEntry(0, 0), EpochEntry(1, 1)), epochCache(leader).epochEntries) + assertEquals(Buffer(EpochEntry(0, 0)), epochCache(follower).epochEntries) //Send a message producer.send(new ProducerRecord(topic, 0, null, msg)).get //Epoch1 should now propagate to the follower with the written message - assertEquals(Buffer(EpochEntry(0, 0), EpochEntry(1, 1)), epochCache(leader).epochEntries()) - assertEquals(Buffer(EpochEntry(0, 0), EpochEntry(1, 1)), epochCache(follower).epochEntries()) + assertEquals(Buffer(EpochEntry(0, 0), EpochEntry(1, 1)), epochCache(leader).epochEntries) + assertEquals(Buffer(EpochEntry(0, 0), EpochEntry(1, 1)), epochCache(follower).epochEntries) //The new message should have epoch 1 stamped assertEquals(1, latestRecord(leader).partitionLeaderEpoch()) @@ -117,8 +117,8 @@ class EpochDrivenReplicationProtocolAcceptanceTest extends ZooKeeperTestHarness awaitISR(tp) //Epochs 2 should be added to the leader, but not on the follower (yet), as there has been no replication. - assertEquals(Buffer(EpochEntry(0, 0), EpochEntry(1, 1)), epochCache(leader).epochEntries()) - assertEquals(Buffer(EpochEntry(0, 0), EpochEntry(1, 1)), epochCache(follower).epochEntries()) + assertEquals(Buffer(EpochEntry(0, 0), EpochEntry(1, 1), EpochEntry(2, 2)), epochCache(leader).epochEntries) + assertEquals(Buffer(EpochEntry(0, 0), EpochEntry(1, 1)), epochCache(follower).epochEntries) //Send a message producer.send(new ProducerRecord(topic, 0, null, msg)).get @@ -128,8 +128,8 @@ class EpochDrivenReplicationProtocolAcceptanceTest extends ZooKeeperTestHarness assertEquals(2, latestRecord(follower).partitionLeaderEpoch()) //The leader epoch files should now match on leader and follower - assertEquals(Buffer(EpochEntry(0, 0), EpochEntry(1, 1), EpochEntry(2, 2)), epochCache(leader).epochEntries()) - assertEquals(Buffer(EpochEntry(0, 0), EpochEntry(1, 1), EpochEntry(2, 2)), epochCache(follower).epochEntries()) + assertEquals(Buffer(EpochEntry(0, 0), EpochEntry(1, 1), EpochEntry(2, 2)), epochCache(leader).epochEntries) + assertEquals(Buffer(EpochEntry(0, 0), EpochEntry(1, 1), EpochEntry(2, 2)), epochCache(follower).epochEntries) } @Test @@ -142,7 +142,7 @@ class EpochDrivenReplicationProtocolAcceptanceTest extends ZooKeeperTestHarness adminZkClient.createOrUpdateTopicPartitionAssignmentPathInZK(topic, Map( 0 -> Seq(100, 101) )) - producer = createProducer() + producer = createProducer //Write 10 messages (0 until 10).foreach { i => @@ -164,7 +164,7 @@ class EpochDrivenReplicationProtocolAcceptanceTest extends ZooKeeperTestHarness //Bounce the producer (this is required, although I'm unsure as to why?) producer.close() - producer = createProducer() + producer = createProducer //Write ten larger messages (so we can easily distinguish between messages written in the two phases) (0 until 10).foreach { _ => @@ -192,7 +192,7 @@ class EpochDrivenReplicationProtocolAcceptanceTest extends ZooKeeperTestHarness adminZkClient.createOrUpdateTopicPartitionAssignmentPathInZK(topic, Map( 0 -> Seq(100, 101) )) - producer = bufferingProducer() + producer = createBufferingProducer //Write 100 messages (0 until 100).foreach { i => @@ -214,7 +214,7 @@ class EpochDrivenReplicationProtocolAcceptanceTest extends ZooKeeperTestHarness //Bounce the producer (this is required, although I'm unsure as to why?) producer.close() - producer = bufferingProducer() + producer = createBufferingProducer //Write two large batches of messages. This will ensure that the LeO of the follower's log aligns with the middle //of the a compressed message set in the leader (which, when forwarded, will result in offsets going backwards) @@ -267,7 +267,7 @@ class EpochDrivenReplicationProtocolAcceptanceTest extends ZooKeeperTestHarness //A single partition topic with 2 replicas adminZkClient.createOrUpdateTopicPartitionAssignmentPathInZK(topic, Map(0 -> Seq(100, 101))) - producer = createProducer() + producer = createProducer //Kick off with a single record producer.send(new ProducerRecord(topic, 0, null, msg)).get @@ -309,7 +309,7 @@ class EpochDrivenReplicationProtocolAcceptanceTest extends ZooKeeperTestHarness adminZkClient.createOrUpdateTopicPartitionAssignmentPathInZK( topic, Map(0 -> Seq(100, 101)), config = CoreUtils.propsWith((KafkaConfig.MinInSyncReplicasProp, "1")) ) - producer = createNewProducer(getBrokerListStrFromServers(brokers), retries = 5, acks = 1) + producer = TestUtils.createProducer(getBrokerListStrFromServers(brokers), retries = 5, acks = 1) // Write one message while both brokers are up (0 until 1).foreach { i => @@ -332,7 +332,7 @@ class EpochDrivenReplicationProtocolAcceptanceTest extends ZooKeeperTestHarness //Bounce the producer (this is required, probably because the broker port changes on restart?) producer.close() - producer = createNewProducer(getBrokerListStrFromServers(brokers), retries = 5, acks = 1) + producer = TestUtils.createProducer(getBrokerListStrFromServers(brokers), retries = 5, acks = 1) //Write 3 messages (0 until 3).foreach { i => @@ -344,7 +344,7 @@ class EpochDrivenReplicationProtocolAcceptanceTest extends ZooKeeperTestHarness //Bounce the producer (this is required, probably because the broker port changes on restart?) producer.close() - producer = createNewProducer(getBrokerListStrFromServers(brokers), retries = 5, acks = 1) + producer = TestUtils.createProducer(getBrokerListStrFromServers(brokers), retries = 5, acks = 1) //Write 1 message (0 until 1).foreach { i => @@ -356,7 +356,7 @@ class EpochDrivenReplicationProtocolAcceptanceTest extends ZooKeeperTestHarness //Bounce the producer (this is required, probably because the broker port changes on restart?) producer.close() - producer = createNewProducer(getBrokerListStrFromServers(brokers), retries = 5, acks = 1) + producer = TestUtils.createProducer(getBrokerListStrFromServers(brokers), retries = 5, acks = 1) //Write 2 messages (0 until 2).foreach { i => @@ -381,8 +381,8 @@ class EpochDrivenReplicationProtocolAcceptanceTest extends ZooKeeperTestHarness private def log(leader: KafkaServer, follower: KafkaServer): Unit = { info(s"Bounce complete for follower ${follower.config.brokerId}") - info(s"Leader: leo${leader.config.brokerId}: " + getLog(leader, 0).logEndOffset + " cache: " + epochCache(leader).epochEntries()) - info(s"Follower: leo${follower.config.brokerId}: " + getLog(follower, 0).logEndOffset + " cache: " + epochCache(follower).epochEntries()) + info(s"Leader: leo${leader.config.brokerId}: " + getLog(leader, 0).logEndOffset + " cache: " + epochCache(leader).epochEntries) + info(s"Follower: leo${follower.config.brokerId}: " + getLog(follower, 0).logEndOffset + " cache: " + epochCache(follower).epochEntries) } private def waitForLogsToMatch(b1: KafkaServer, b2: KafkaServer, partition: Int = 0): Unit = { @@ -414,8 +414,8 @@ class EpochDrivenReplicationProtocolAcceptanceTest extends ZooKeeperTestHarness writable.close() } - private def bufferingProducer(): KafkaProducer[Array[Byte], Array[Byte]] = { - createNewProducer(getBrokerListStrFromServers(brokers), retries = 5, acks = -1, lingerMs = 10000, + private def createBufferingProducer: KafkaProducer[Array[Byte], Array[Byte]] = { + TestUtils.createProducer(getBrokerListStrFromServers(brokers), retries = 5, acks = -1, lingerMs = 10000, props = Option(CoreUtils.propsWith( (ProducerConfig.BATCH_SIZE_CONFIG, String.valueOf(msg.length * 1000)) , (ProducerConfig.COMPRESSION_TYPE_CONFIG, "snappy") @@ -436,7 +436,7 @@ class EpochDrivenReplicationProtocolAcceptanceTest extends ZooKeeperTestHarness follower.shutdown() follower.startup() producer.close() - producer = createProducer() //TODO not sure why we need to recreate the producer, but it doesn't reconnect if we don't + producer = createProducer //TODO not sure why we need to recreate the producer, but it doesn't reconnect if we don't } private def epochCache(broker: KafkaServer): LeaderEpochFileCache = { @@ -454,8 +454,8 @@ class EpochDrivenReplicationProtocolAcceptanceTest extends ZooKeeperTestHarness }, "Timed out waiting for replicas to join ISR") } - private def createProducer(): KafkaProducer[Array[Byte], Array[Byte]] = { - createNewProducer(getBrokerListStrFromServers(brokers), retries = 5, acks = -1) + private def createProducer: KafkaProducer[Array[Byte], Array[Byte]] = { + TestUtils.createProducer(getBrokerListStrFromServers(brokers), retries = 5, acks = -1) } private def leader(): KafkaServer = { diff --git a/core/src/test/scala/unit/kafka/server/epoch/LeaderEpochFileCacheTest.scala b/core/src/test/scala/unit/kafka/server/epoch/LeaderEpochFileCacheTest.scala index d1f93900ccf93..7ac606a2dc5d0 100644 --- a/core/src/test/scala/unit/kafka/server/epoch/LeaderEpochFileCacheTest.scala +++ b/core/src/test/scala/unit/kafka/server/epoch/LeaderEpochFileCacheTest.scala @@ -16,6 +16,7 @@ */ package kafka.server.epoch + import java.io.File import kafka.server.LogOffsetMetadata @@ -24,7 +25,7 @@ import org.apache.kafka.common.requests.EpochEndOffset.{UNDEFINED_EPOCH, UNDEFIN import kafka.utils.TestUtils import org.apache.kafka.common.TopicPartition import org.junit.Assert._ -import org.junit.{Before, Test} +import org.junit.Test import scala.collection.mutable.ListBuffer @@ -33,54 +34,44 @@ import scala.collection.mutable.ListBuffer */ class LeaderEpochFileCacheTest { val tp = new TopicPartition("TestTopic", 5) - var checkpoint: LeaderEpochCheckpoint = _ + private var logEndOffset = 0L + private val checkpoint: LeaderEpochCheckpoint = new LeaderEpochCheckpoint { + private var epochs: Seq[EpochEntry] = Seq() + override def write(epochs: Seq[EpochEntry]): Unit = this.epochs = epochs + override def read(): Seq[EpochEntry] = this.epochs + } + private val cache = new LeaderEpochFileCache(tp, logEndOffset _, checkpoint) @Test def shouldAddEpochAndMessageOffsetToCache() = { - var leo = 0 - def leoFinder() = new LogOffsetMetadata(leo) - - //Given - val cache = new LeaderEpochFileCache(tp, () => leoFinder, checkpoint) - //When - cache.assign(epoch = 2, offset = 10) - leo = 11 + cache.assign(epoch = 2, startOffset = 10) + logEndOffset = 11 //Then - assertEquals(2, cache.latestEpoch()) - assertEquals(EpochEntry(2, 10), cache.epochEntries()(0)) - assertEquals((2, leo), cache.endOffsetFor(2)) //should match leo + assertEquals(2, cache.latestEpoch) + assertEquals(EpochEntry(2, 10), cache.epochEntries(0)) + assertEquals((2, logEndOffset), cache.endOffsetFor(2)) //should match logEndOffset } @Test def shouldReturnLogEndOffsetIfLatestEpochRequested() = { - var leo = 0 - def leoFinder() = new LogOffsetMetadata(leo) - - //Given - val cache = new LeaderEpochFileCache(tp, () => leoFinder, checkpoint) - //When just one epoch - cache.assign(epoch = 2, offset = 11) - cache.assign(epoch = 2, offset = 12) - leo = 14 + cache.assign(epoch = 2, startOffset = 11) + cache.assign(epoch = 2, startOffset = 12) + logEndOffset = 14 //Then - assertEquals((2, leo), cache.endOffsetFor(2)) + assertEquals((2, logEndOffset), cache.endOffsetFor(2)) } @Test def shouldReturnUndefinedOffsetIfUndefinedEpochRequested() = { - def leoFinder() = new LogOffsetMetadata(0) val expectedEpochEndOffset = (UNDEFINED_EPOCH, UNDEFINED_EPOCH_OFFSET) - //Given cache with some data on leader - val cache = new LeaderEpochFileCache(tp, () => leoFinder, checkpoint) - // assign couple of epochs - cache.assign(epoch = 2, offset = 11) - cache.assign(epoch = 3, offset = 12) + cache.assign(epoch = 2, startOffset = 11) + cache.assign(epoch = 3, startOffset = 12) //When (say a bootstraping follower) sends request for UNDEFINED_EPOCH val epochAndOffsetFor = cache.endOffsetFor(UNDEFINED_EPOCH) @@ -92,68 +83,51 @@ class LeaderEpochFileCacheTest { @Test def shouldNotOverwriteLogEndOffsetForALeaderEpochOnceItHasBeenAssigned() = { - var leo = 0 - def leoFinder() = new LogOffsetMetadata(leo) - //Given - leo = 9 - val cache = new LeaderEpochFileCache(tp, () => leoFinder, checkpoint) + logEndOffset = 9 - cache.assign(2, leo) + cache.assign(2, logEndOffset) //When called again later cache.assign(2, 10) //Then the offset should NOT have been updated - assertEquals(leo, cache.epochEntries()(0).startOffset) + assertEquals(logEndOffset, cache.epochEntries(0).startOffset) + assertEquals(ListBuffer(EpochEntry(2, 9)), cache.epochEntries) } @Test - def shouldAllowLeaderEpochToChangeEvenIfOffsetDoesNot() = { - def leoFinder() = new LogOffsetMetadata(0) - + def shouldEnforceMonotonicallyIncreasingStartOffsets() = { //Given - val cache = new LeaderEpochFileCache(tp, () => leoFinder, checkpoint) cache.assign(2, 9) //When update epoch new epoch but same offset cache.assign(3, 9) //Then epoch should have been updated - assertEquals(ListBuffer(EpochEntry(2, 9), EpochEntry(3, 9)), cache.epochEntries()) + assertEquals(ListBuffer(EpochEntry(3, 9)), cache.epochEntries) } @Test def shouldNotOverwriteOffsetForALeaderEpochOnceItHasBeenAssigned() = { - //Given - val cache = new LeaderEpochFileCache(tp, () => new LogOffsetMetadata(0), checkpoint) cache.assign(2, 6) //When called again later with a greater offset cache.assign(2, 10) //Then later update should have been ignored - assertEquals(6, cache.epochEntries()(0).startOffset) + assertEquals(6, cache.epochEntries(0).startOffset) } @Test def shouldReturnUnsupportedIfNoEpochRecorded(){ - def leoFinder() = new LogOffsetMetadata(0) - - //Given - val cache = new LeaderEpochFileCache(tp, () => leoFinder, checkpoint) - //Then assertEquals((UNDEFINED_EPOCH, UNDEFINED_EPOCH_OFFSET), cache.endOffsetFor(0)) } @Test def shouldReturnUnsupportedIfNoEpochRecordedAndUndefinedEpochRequested(){ - val leo = 73 - def leoFinder() = new LogOffsetMetadata(leo) - - //Given - val cache = new LeaderEpochFileCache(tp, () => leoFinder, checkpoint) + logEndOffset = 73 //When (say a follower on older message format version) sends request for UNDEFINED_EPOCH val offsetFor = cache.endOffsetFor(UNDEFINED_EPOCH) @@ -164,39 +138,41 @@ class LeaderEpochFileCacheTest { } @Test - def shouldReturnUnsupportedIfRequestedEpochLessThanFirstEpoch(){ - def leoFinder() = new LogOffsetMetadata(0) - - //Given - val cache = new LeaderEpochFileCache(tp, () => leoFinder, checkpoint) - - cache.assign(epoch = 5, offset = 11) - cache.assign(epoch = 6, offset = 12) - cache.assign(epoch = 7, offset = 13) + def shouldReturnFirstEpochIfRequestedEpochLessThanFirstEpoch(){ + cache.assign(epoch = 5, startOffset = 11) + cache.assign(epoch = 6, startOffset = 12) + cache.assign(epoch = 7, startOffset = 13) //When - val epochAndOffset = cache.endOffsetFor(5 - 1) + val epochAndOffset = cache.endOffsetFor(4) //Then - assertEquals((UNDEFINED_EPOCH, UNDEFINED_EPOCH_OFFSET), epochAndOffset) + assertEquals((4, 11), epochAndOffset) } @Test - def shouldGetFirstOffsetOfSubsequentEpochWhenOffsetRequestedForPreviousEpoch() = { - var leo = 0 - def leoFinder() = new LogOffsetMetadata(leo) + def shouldTruncateIfMatchingEpochButEarlierStartingOffset(): Unit = { + cache.assign(epoch = 5, startOffset = 11) + cache.assign(epoch = 6, startOffset = 12) + cache.assign(epoch = 7, startOffset = 13) - //Given - val cache = new LeaderEpochFileCache(tp, () => leoFinder, checkpoint) + // epoch 7 starts at an earlier offset + cache.assign(epoch = 7, startOffset = 12) + assertEquals((5, 12), cache.endOffsetFor(5)) + assertEquals((5, 12), cache.endOffsetFor(6)) + } + + @Test + def shouldGetFirstOffsetOfSubsequentEpochWhenOffsetRequestedForPreviousEpoch() = { //When several epochs - cache.assign(epoch = 1, offset = 11) - cache.assign(epoch = 1, offset = 12) - cache.assign(epoch = 2, offset = 13) - cache.assign(epoch = 2, offset = 14) - cache.assign(epoch = 3, offset = 15) - cache.assign(epoch = 3, offset = 16) - leo = 17 + cache.assign(epoch = 1, startOffset = 11) + cache.assign(epoch = 1, startOffset = 12) + cache.assign(epoch = 2, startOffset = 13) + cache.assign(epoch = 2, startOffset = 14) + cache.assign(epoch = 3, startOffset = 15) + cache.assign(epoch = 3, startOffset = 16) + logEndOffset = 17 //Then get the start offset of the next epoch assertEquals((2, 15), cache.endOffsetFor(2)) @@ -204,15 +180,10 @@ class LeaderEpochFileCacheTest { @Test def shouldReturnNextAvailableEpochIfThereIsNoExactEpochForTheOneRequested(){ - def leoFinder() = new LogOffsetMetadata(0) - - //Given - val cache = new LeaderEpochFileCache(tp, () => leoFinder, checkpoint) - //When - cache.assign(epoch = 0, offset = 10) - cache.assign(epoch = 2, offset = 13) - cache.assign(epoch = 4, offset = 17) + cache.assign(epoch = 0, startOffset = 10) + cache.assign(epoch = 2, startOffset = 13) + cache.assign(epoch = 4, startOffset = 17) //Then assertEquals((0, 13), cache.endOffsetFor(requestedEpoch = 1)) @@ -222,14 +193,9 @@ class LeaderEpochFileCacheTest { @Test def shouldNotUpdateEpochAndStartOffsetIfItDidNotChange() = { - def leoFinder() = new LogOffsetMetadata(0) - - //Given - val cache = new LeaderEpochFileCache(tp, () => leoFinder, checkpoint) - //When - cache.assign(epoch = 2, offset = 6) - cache.assign(epoch = 2, offset = 7) + cache.assign(epoch = 2, startOffset = 6) + cache.assign(epoch = 2, startOffset = 7) //Then assertEquals(1, cache.epochEntries.size) @@ -238,14 +204,10 @@ class LeaderEpochFileCacheTest { @Test def shouldReturnInvalidOffsetIfEpochIsRequestedWhichIsNotCurrentlyTracked(): Unit = { - val leo = 100 - def leoFinder() = new LogOffsetMetadata(leo) - - //Given - val cache = new LeaderEpochFileCache(tp, () => leoFinder, checkpoint) + logEndOffset = 100 //When - cache.assign(epoch = 2, offset = 100) + cache.assign(epoch = 2, startOffset = 100) //Then assertEquals((UNDEFINED_EPOCH, UNDEFINED_EPOCH_OFFSET), cache.endOffsetFor(3)) @@ -253,35 +215,28 @@ class LeaderEpochFileCacheTest { @Test def shouldSupportEpochsThatDoNotStartFromZero(): Unit = { - var leo = 0 - def leoFinder() = new LogOffsetMetadata(leo) - - //Given - val cache = new LeaderEpochFileCache(tp, () => leoFinder, checkpoint) - //When - cache.assign(epoch = 2, offset = 6) - leo = 7 + cache.assign(epoch = 2, startOffset = 6) + logEndOffset = 7 //Then - assertEquals((2, leo), cache.endOffsetFor(2)) + assertEquals((2, logEndOffset), cache.endOffsetFor(2)) assertEquals(1, cache.epochEntries.size) - assertEquals(EpochEntry(2, 6), cache.epochEntries()(0)) + assertEquals(EpochEntry(2, 6), cache.epochEntries(0)) } @Test def shouldPersistEpochsBetweenInstances(){ - def leoFinder() = new LogOffsetMetadata(0) val checkpointPath = TestUtils.tempFile().getAbsolutePath - checkpoint = new LeaderEpochCheckpointFile(new File(checkpointPath)) + val checkpoint = new LeaderEpochCheckpointFile(new File(checkpointPath)) //Given - val cache = new LeaderEpochFileCache(tp, () => leoFinder, checkpoint) - cache.assign(epoch = 2, offset = 6) + val cache = new LeaderEpochFileCache(tp, logEndOffset _, checkpoint) + cache.assign(epoch = 2, startOffset = 6) //When val checkpoint2 = new LeaderEpochCheckpointFile(new File(checkpointPath)) - val cache2 = new LeaderEpochFileCache(tp, () => leoFinder, checkpoint2) + val cache2 = new LeaderEpochFileCache(tp, logEndOffset _, checkpoint2) //Then assertEquals(1, cache2.epochEntries.size) @@ -289,81 +244,68 @@ class LeaderEpochFileCacheTest { } @Test - def shouldNotLetEpochGoBackwardsEvenIfMessageEpochsDo(): Unit = { - var leo = 0 - def leoFinder() = new LogOffsetMetadata(leo) - - val cache = new LeaderEpochFileCache(tp, () => leoFinder, checkpoint) - + def shouldEnforceMonotonicallyIncreasingEpochs(): Unit = { //Given - cache.assign(epoch = 1, offset = 5); leo = 6 - cache.assign(epoch = 2, offset = 6); leo = 7 - - //When we update an epoch in the past with an earlier offset - cache.assign(epoch = 1, offset = 7); leo = 8 + cache.assign(epoch = 1, startOffset = 5); logEndOffset = 6 + cache.assign(epoch = 2, startOffset = 6); logEndOffset = 7 - //Then epoch should not be changed - assertEquals(2, cache.latestEpoch()) + //When we update an epoch in the past with a different offset, the log has already reached + //an inconsistent state. Our options are either to raise an error, ignore the new append, + //or truncate the cached epochs to the point of conflict. We take this latter approach in + //order to guarantee that epochs and offsets in the cache increase monotonically, which makes + //the search logic simpler to reason about. + cache.assign(epoch = 1, startOffset = 7); logEndOffset = 8 - //Then end offset for epoch 1 shouldn't have changed - assertEquals((1, 6), cache.endOffsetFor(1)) + //Then later epochs will be removed + assertEquals(1, cache.latestEpoch) - //Then end offset for epoch 2 has to be the offset of the epoch 1 message (I can't think of a better option) - assertEquals((2, 8), cache.endOffsetFor(2)) + //Then end offset for epoch 1 will have changed + assertEquals((1, 8), cache.endOffsetFor(1)) - //Epoch history shouldn't have changed - assertEquals(EpochEntry(1, 5), cache.epochEntries()(0)) - assertEquals(EpochEntry(2, 6), cache.epochEntries()(1)) + //Then end offset for epoch 2 is now undefined + assertEquals((UNDEFINED_EPOCH, UNDEFINED_EPOCH_OFFSET), cache.endOffsetFor(2)) + assertEquals(EpochEntry(1, 7), cache.epochEntries(0)) } @Test - def shouldNotLetOffsetsGoBackwardsEvenIfEpochsProgress() = { - def leoFinder() = new LogOffsetMetadata(0) - - //Given - val cache = new LeaderEpochFileCache(tp, () => leoFinder, checkpoint) - + def shouldEnforceOffsetsIncreaseMonotonically() = { //When epoch goes forward but offset goes backwards - cache.assign(epoch = 2, offset = 6) - cache.assign(epoch = 3, offset = 5) + cache.assign(epoch = 2, startOffset = 6) + cache.assign(epoch = 3, startOffset = 5) - //Then latter assign should be ignored - assertEquals(EpochEntry(2, 6), cache.epochEntries.toList(0)) + //The last assignment wins and the conflicting one is removed from the log + assertEquals(EpochEntry(3, 5), cache.epochEntries.toList(0)) } @Test def shouldIncreaseAndTrackEpochsAsLeadersChangeManyTimes(): Unit = { - var leo = 0 - def leoFinder() = new LogOffsetMetadata(leo) - //Given - val cache = new LeaderEpochFileCache(tp, () => leoFinder, checkpoint) - cache.assign(epoch = 0, offset = 0) //leo=0 + cache.assign(epoch = 0, startOffset = 0) //logEndOffset=0 //When - cache.assign(epoch = 1, offset = 0) //leo=0 + cache.assign(epoch = 1, startOffset = 0) //logEndOffset=0 //Then epoch should go up - assertEquals(1, cache.latestEpoch()) + assertEquals(1, cache.latestEpoch) //offset for 1 should still be 0 assertEquals((1, 0), cache.endOffsetFor(1)) //offset for epoch 0 should still be 0 assertEquals((0, 0), cache.endOffsetFor(0)) //When we write 5 messages as epoch 1 - leo = 5 + logEndOffset = 5 - //Then end offset for epoch(1) should be leo => 5 + //Then end offset for epoch(1) should be logEndOffset => 5 assertEquals((1, 5), cache.endOffsetFor(1)) //Epoch 0 should still be at offset 0 assertEquals((0, 0), cache.endOffsetFor(0)) //When - cache.assign(epoch = 2, offset = 5) //leo=5 + cache.assign(epoch = 2, startOffset = 5) //logEndOffset=5 - leo = 10 //write another 5 messages + logEndOffset = 10 //write another 5 messages - //Then end offset for epoch(2) should be leo => 10 + //Then end offset for epoch(2) should be logEndOffset => 10 assertEquals((2, 10), cache.endOffsetFor(2)) //end offset for epoch(1) should be the start offset of epoch(2) => 5 @@ -375,36 +317,30 @@ class LeaderEpochFileCacheTest { @Test def shouldIncreaseAndTrackEpochsAsFollowerReceivesManyMessages(): Unit = { - var leo = 0 - def leoFinder() = new LogOffsetMetadata(leo) - - //When new - val cache = new LeaderEpochFileCache(tp, () => leoFinder, checkpoint) - //When Messages come in - cache.assign(epoch = 0, offset = 0); leo = 1 - cache.assign(epoch = 0, offset = 1); leo = 2 - cache.assign(epoch = 0, offset = 2); leo = 3 + cache.assign(epoch = 0, startOffset = 0); logEndOffset = 1 + cache.assign(epoch = 0, startOffset = 1); logEndOffset = 2 + cache.assign(epoch = 0, startOffset = 2); logEndOffset = 3 //Then epoch should stay, offsets should grow - assertEquals(0, cache.latestEpoch()) - assertEquals((0, leo), cache.endOffsetFor(0)) + assertEquals(0, cache.latestEpoch) + assertEquals((0, logEndOffset), cache.endOffsetFor(0)) //When messages arrive with greater epoch - cache.assign(epoch = 1, offset = 3); leo = 4 - cache.assign(epoch = 1, offset = 4); leo = 5 - cache.assign(epoch = 1, offset = 5); leo = 6 + cache.assign(epoch = 1, startOffset = 3); logEndOffset = 4 + cache.assign(epoch = 1, startOffset = 4); logEndOffset = 5 + cache.assign(epoch = 1, startOffset = 5); logEndOffset = 6 - assertEquals(1, cache.latestEpoch()) - assertEquals((1, leo), cache.endOffsetFor(1)) + assertEquals(1, cache.latestEpoch) + assertEquals((1, logEndOffset), cache.endOffsetFor(1)) //When - cache.assign(epoch = 2, offset = 6); leo = 7 - cache.assign(epoch = 2, offset = 7); leo = 8 - cache.assign(epoch = 2, offset = 8); leo = 9 + cache.assign(epoch = 2, startOffset = 6); logEndOffset = 7 + cache.assign(epoch = 2, startOffset = 7); logEndOffset = 8 + cache.assign(epoch = 2, startOffset = 8); logEndOffset = 9 - assertEquals(2, cache.latestEpoch()) - assertEquals((2, leo), cache.endOffsetFor(2)) + assertEquals(2, cache.latestEpoch) + assertEquals((2, logEndOffset), cache.endOffsetFor(2)) //Older epochs should return the start offset of the first message in the subsequent epoch. assertEquals((0, 3), cache.endOffsetFor(0)) @@ -413,16 +349,13 @@ class LeaderEpochFileCacheTest { @Test def shouldDropEntriesOnEpochBoundaryWhenRemovingLatestEntries(): Unit = { - def leoFinder() = new LogOffsetMetadata(0) - //Given - val cache = new LeaderEpochFileCache(tp, () => leoFinder, checkpoint) - cache.assign(epoch = 2, offset = 6) - cache.assign(epoch = 3, offset = 8) - cache.assign(epoch = 4, offset = 11) + cache.assign(epoch = 2, startOffset = 6) + cache.assign(epoch = 3, startOffset = 8) + cache.assign(epoch = 4, startOffset = 11) //When clear latest on epoch boundary - cache.clearAndFlushLatest(offset = 8) + cache.truncateFromEnd(endOffset = 8) //Then should remove two latest epochs (remove is inclusive) assertEquals(ListBuffer(EpochEntry(2, 6)), cache.epochEntries) @@ -430,16 +363,13 @@ class LeaderEpochFileCacheTest { @Test def shouldPreserveResetOffsetOnClearEarliestIfOneExists(): Unit = { - def leoFinder() = new LogOffsetMetadata(0) - //Given - val cache = new LeaderEpochFileCache(tp, () => leoFinder, checkpoint) - cache.assign(epoch = 2, offset = 6) - cache.assign(epoch = 3, offset = 8) - cache.assign(epoch = 4, offset = 11) + cache.assign(epoch = 2, startOffset = 6) + cache.assign(epoch = 3, startOffset = 8) + cache.assign(epoch = 4, startOffset = 11) //When reset to offset ON epoch boundary - cache.clearAndFlushEarliest(offset = 8) + cache.truncateFromStart(startOffset = 8) //Then should preserve (3, 8) assertEquals(ListBuffer(EpochEntry(3, 8), EpochEntry(4, 11)), cache.epochEntries) @@ -447,16 +377,13 @@ class LeaderEpochFileCacheTest { @Test def shouldUpdateSavedOffsetWhenOffsetToClearToIsBetweenEpochs(): Unit = { - def leoFinder() = new LogOffsetMetadata(0) - //Given - val cache = new LeaderEpochFileCache(tp, () => leoFinder, checkpoint) - cache.assign(epoch = 2, offset = 6) - cache.assign(epoch = 3, offset = 8) - cache.assign(epoch = 4, offset = 11) + cache.assign(epoch = 2, startOffset = 6) + cache.assign(epoch = 3, startOffset = 8) + cache.assign(epoch = 4, startOffset = 11) //When reset to offset BETWEEN epoch boundaries - cache.clearAndFlushEarliest(offset = 9) + cache.truncateFromStart(startOffset = 9) //Then we should retain epoch 3, but update it's offset to 9 as 8 has been removed assertEquals(ListBuffer(EpochEntry(3, 9), EpochEntry(4, 11)), cache.epochEntries) @@ -464,16 +391,13 @@ class LeaderEpochFileCacheTest { @Test def shouldNotClearAnythingIfOffsetToEarly(): Unit = { - def leoFinder() = new LogOffsetMetadata(0) - //Given - val cache = new LeaderEpochFileCache(tp, () => leoFinder, checkpoint) - cache.assign(epoch = 2, offset = 6) - cache.assign(epoch = 3, offset = 8) - cache.assign(epoch = 4, offset = 11) + cache.assign(epoch = 2, startOffset = 6) + cache.assign(epoch = 3, startOffset = 8) + cache.assign(epoch = 4, startOffset = 11) //When reset to offset before first epoch offset - cache.clearAndFlushEarliest(offset = 1) + cache.truncateFromStart(startOffset = 1) //Then nothing should change assertEquals(ListBuffer(EpochEntry(2, 6),EpochEntry(3, 8), EpochEntry(4, 11)), cache.epochEntries) @@ -481,16 +405,13 @@ class LeaderEpochFileCacheTest { @Test def shouldNotClearAnythingIfOffsetToFirstOffset(): Unit = { - def leoFinder() = new LogOffsetMetadata(0) - //Given - val cache = new LeaderEpochFileCache(tp, () => leoFinder, checkpoint) - cache.assign(epoch = 2, offset = 6) - cache.assign(epoch = 3, offset = 8) - cache.assign(epoch = 4, offset = 11) + cache.assign(epoch = 2, startOffset = 6) + cache.assign(epoch = 3, startOffset = 8) + cache.assign(epoch = 4, startOffset = 11) //When reset to offset on earliest epoch boundary - cache.clearAndFlushEarliest(offset = 6) + cache.truncateFromStart(startOffset = 6) //Then nothing should change assertEquals(ListBuffer(EpochEntry(2, 6),EpochEntry(3, 8), EpochEntry(4, 11)), cache.epochEntries) @@ -498,16 +419,13 @@ class LeaderEpochFileCacheTest { @Test def shouldRetainLatestEpochOnClearAllEarliest(): Unit = { - def leoFinder() = new LogOffsetMetadata(0) - //Given - val cache = new LeaderEpochFileCache(tp, () => leoFinder, checkpoint) - cache.assign(epoch = 2, offset = 6) - cache.assign(epoch = 3, offset = 8) - cache.assign(epoch = 4, offset = 11) + cache.assign(epoch = 2, startOffset = 6) + cache.assign(epoch = 3, startOffset = 8) + cache.assign(epoch = 4, startOffset = 11) //When - cache.clearAndFlushEarliest(offset = 11) + cache.truncateFromStart(startOffset = 11) //Then retain the last assertEquals(ListBuffer(EpochEntry(4, 11)), cache.epochEntries) @@ -515,16 +433,13 @@ class LeaderEpochFileCacheTest { @Test def shouldUpdateOffsetBetweenEpochBoundariesOnClearEarliest(): Unit = { - def leoFinder() = new LogOffsetMetadata(0) - //Given - val cache = new LeaderEpochFileCache(tp, () => leoFinder, checkpoint) - cache.assign(epoch = 2, offset = 6) - cache.assign(epoch = 3, offset = 8) - cache.assign(epoch = 4, offset = 11) + cache.assign(epoch = 2, startOffset = 6) + cache.assign(epoch = 3, startOffset = 8) + cache.assign(epoch = 4, startOffset = 11) //When we clear from a postition between offset 8 & offset 11 - cache.clearAndFlushEarliest(offset = 9) + cache.truncateFromStart(startOffset = 9) //Then we should update the middle epoch entry's offset assertEquals(ListBuffer(EpochEntry(3, 9), EpochEntry(4, 11)), cache.epochEntries) @@ -532,16 +447,13 @@ class LeaderEpochFileCacheTest { @Test def shouldUpdateOffsetBetweenEpochBoundariesOnClearEarliest2(): Unit = { - def leoFinder() = new LogOffsetMetadata(0) - //Given - val cache = new LeaderEpochFileCache(tp, () => leoFinder, checkpoint) - cache.assign(epoch = 0, offset = 0) - cache.assign(epoch = 1, offset = 7) - cache.assign(epoch = 2, offset = 10) + cache.assign(epoch = 0, startOffset = 0) + cache.assign(epoch = 1, startOffset = 7) + cache.assign(epoch = 2, startOffset = 10) //When we clear from a postition between offset 0 & offset 7 - cache.clearAndFlushEarliest(offset = 5) + cache.truncateFromStart(startOffset = 5) //Then we should keeep epoch 0 but update the offset appropriately assertEquals(ListBuffer(EpochEntry(0,5), EpochEntry(1, 7), EpochEntry(2, 10)), cache.epochEntries) @@ -549,16 +461,13 @@ class LeaderEpochFileCacheTest { @Test def shouldRetainLatestEpochOnClearAllEarliestAndUpdateItsOffset(): Unit = { - def leoFinder() = new LogOffsetMetadata(0) - //Given - val cache = new LeaderEpochFileCache(tp, () => leoFinder, checkpoint) - cache.assign(epoch = 2, offset = 6) - cache.assign(epoch = 3, offset = 8) - cache.assign(epoch = 4, offset = 11) + cache.assign(epoch = 2, startOffset = 6) + cache.assign(epoch = 3, startOffset = 8) + cache.assign(epoch = 4, startOffset = 11) //When reset to offset beyond last epoch - cache.clearAndFlushEarliest(offset = 15) + cache.truncateFromStart(startOffset = 15) //Then update the last assertEquals(ListBuffer(EpochEntry(4, 15)), cache.epochEntries) @@ -566,51 +475,42 @@ class LeaderEpochFileCacheTest { @Test def shouldDropEntriesBetweenEpochBoundaryWhenRemovingNewest(): Unit = { - def leoFinder() = new LogOffsetMetadata(0) - //Given - val cache = new LeaderEpochFileCache(tp, () => leoFinder, checkpoint) - cache.assign(epoch = 2, offset = 6) - cache.assign(epoch = 3, offset = 8) - cache.assign(epoch = 4, offset = 11) + cache.assign(epoch = 2, startOffset = 6) + cache.assign(epoch = 3, startOffset = 8) + cache.assign(epoch = 4, startOffset = 11) //When reset to offset BETWEEN epoch boundaries - cache.clearAndFlushLatest(offset = 9) + cache.truncateFromEnd(endOffset = 9) //Then should keep the preceding epochs - assertEquals(3, cache.latestEpoch()) + assertEquals(3, cache.latestEpoch) assertEquals(ListBuffer(EpochEntry(2, 6), EpochEntry(3, 8)), cache.epochEntries) } @Test def shouldClearAllEntries(): Unit = { - def leoFinder() = new LogOffsetMetadata(0) - //Given - val cache = new LeaderEpochFileCache(tp, () => leoFinder, checkpoint) - cache.assign(epoch = 2, offset = 6) - cache.assign(epoch = 3, offset = 8) - cache.assign(epoch = 4, offset = 11) + cache.assign(epoch = 2, startOffset = 6) + cache.assign(epoch = 3, startOffset = 8) + cache.assign(epoch = 4, startOffset = 11) - //When + //When cache.clearAndFlush() - //Then + //Then assertEquals(0, cache.epochEntries.size) } @Test def shouldNotResetEpochHistoryHeadIfUndefinedPassed(): Unit = { - def leoFinder() = new LogOffsetMetadata(0) - //Given - val cache = new LeaderEpochFileCache(tp, () => leoFinder, checkpoint) - cache.assign(epoch = 2, offset = 6) - cache.assign(epoch = 3, offset = 8) - cache.assign(epoch = 4, offset = 11) + cache.assign(epoch = 2, startOffset = 6) + cache.assign(epoch = 3, startOffset = 8) + cache.assign(epoch = 4, startOffset = 11) //When reset to offset on epoch boundary - cache.clearAndFlushLatest(offset = UNDEFINED_EPOCH_OFFSET) + cache.truncateFromEnd(endOffset = UNDEFINED_EPOCH_OFFSET) //Then should do nothing assertEquals(3, cache.epochEntries.size) @@ -618,16 +518,13 @@ class LeaderEpochFileCacheTest { @Test def shouldNotResetEpochHistoryTailIfUndefinedPassed(): Unit = { - def leoFinder() = new LogOffsetMetadata(0) - //Given - val cache = new LeaderEpochFileCache(tp, () => leoFinder, checkpoint) - cache.assign(epoch = 2, offset = 6) - cache.assign(epoch = 3, offset = 8) - cache.assign(epoch = 4, offset = 11) + cache.assign(epoch = 2, startOffset = 6) + cache.assign(epoch = 3, startOffset = 8) + cache.assign(epoch = 4, startOffset = 11) //When reset to offset on epoch boundary - cache.clearAndFlushEarliest(offset = UNDEFINED_EPOCH_OFFSET) + cache.truncateFromEnd(endOffset = UNDEFINED_EPOCH_OFFSET) //Then should do nothing assertEquals(3, cache.epochEntries.size) @@ -635,54 +532,26 @@ class LeaderEpochFileCacheTest { @Test def shouldFetchLatestEpochOfEmptyCache(): Unit = { - //Given - def leoFinder() = new LogOffsetMetadata(0) - - //When - val cache = new LeaderEpochFileCache(tp, () => leoFinder, checkpoint) - //Then assertEquals(-1, cache.latestEpoch) } @Test def shouldFetchEndOffsetOfEmptyCache(): Unit = { - //Given - def leoFinder() = new LogOffsetMetadata(0) - - //When - val cache = new LeaderEpochFileCache(tp, () => leoFinder, checkpoint) - //Then assertEquals((UNDEFINED_EPOCH, UNDEFINED_EPOCH_OFFSET), cache.endOffsetFor(7)) } @Test def shouldClearEarliestOnEmptyCache(): Unit = { - //Given - def leoFinder() = new LogOffsetMetadata(0) - - //When - val cache = new LeaderEpochFileCache(tp, () => leoFinder, checkpoint) - //Then - cache.clearAndFlushEarliest(7) + cache.truncateFromStart(7) } @Test def shouldClearLatestOnEmptyCache(): Unit = { - //Given - def leoFinder() = new LogOffsetMetadata(0) - - //When - val cache = new LeaderEpochFileCache(tp, () => leoFinder, checkpoint) - //Then - cache.clearAndFlushLatest(7) + cache.truncateFromEnd(7) } - @Before - def setUp() { - checkpoint = new LeaderEpochCheckpointFile(TestUtils.tempFile()) - } } diff --git a/core/src/test/scala/unit/kafka/server/epoch/LeaderEpochIntegrationTest.scala b/core/src/test/scala/unit/kafka/server/epoch/LeaderEpochIntegrationTest.scala index 907db7a71fa8b..5ac0d9913fe80 100644 --- a/core/src/test/scala/unit/kafka/server/epoch/LeaderEpochIntegrationTest.scala +++ b/core/src/test/scala/unit/kafka/server/epoch/LeaderEpochIntegrationTest.scala @@ -38,9 +38,10 @@ import org.apache.kafka.common.requests.{EpochEndOffset, OffsetsForLeaderEpochRe import scala.collection.JavaConverters._ import scala.collection.Map +import scala.collection.mutable.ListBuffer class LeaderEpochIntegrationTest extends ZooKeeperTestHarness with Logging { - var brokers: Seq[KafkaServer] = null + var brokers: ListBuffer[KafkaServer] = ListBuffer() val topic1 = "foo" val topic2 = "bar" val t1p0 = new TopicPartition(topic1, 0) @@ -61,7 +62,7 @@ class LeaderEpochIntegrationTest extends ZooKeeperTestHarness with Logging { @Test def shouldAddCurrentLeaderEpochToMessagesAsTheyAreWrittenToLeader() { - brokers = (0 to 1).map { id => createServer(fromProps(createBrokerConfig(id, zkConnect))) } + brokers ++= (0 to 1).map { id => createServer(fromProps(createBrokerConfig(id, zkConnect))) } // Given two topics with replication of a single partition for (topic <- List(topic1, topic2)) { @@ -95,17 +96,16 @@ class LeaderEpochIntegrationTest extends ZooKeeperTestHarness with Logging { def shouldSendLeaderEpochRequestAndGetAResponse(): Unit = { //3 brokers, put partition on 100/101 and then pretend to be 102 - brokers = (100 to 102).map { id => createServer(fromProps(createBrokerConfig(id, zkConnect))) } - adminZkClient.createOrUpdateTopicPartitionAssignmentPathInZK(topic1, Map( - 0 -> Seq(100), - 1 -> Seq(101) - )) - adminZkClient.createOrUpdateTopicPartitionAssignmentPathInZK(topic2, Map( - 0 -> Seq(100) - )) + brokers ++= (100 to 102).map { id => createServer(fromProps(createBrokerConfig(id, zkConnect))) } + + val assignment1 = Map(0 -> Seq(100), 1 -> Seq(101)) + TestUtils.createTopic(zkClient, topic1, assignment1, brokers) + + val assignment2 = Map(0 -> Seq(100)) + TestUtils.createTopic(zkClient, topic2, assignment2, brokers) //Send messages equally to the two partitions, then half as many to a third - producer = createNewProducer(getBrokerListStrFromServers(brokers), retries = 5, acks = -1) + producer = createProducer(getBrokerListStrFromServers(brokers), retries = 5, acks = -1) (0 until 10).foreach { _ => producer.send(new ProducerRecord(topic1, 0, null, "IHeartLogs".getBytes)) } @@ -140,22 +140,25 @@ class LeaderEpochIntegrationTest extends ZooKeeperTestHarness with Logging { @Test def shouldIncreaseLeaderEpochBetweenLeaderRestarts(): Unit = { - //Setup: we are only interested in the single partition on broker 101 - brokers = Seq(100, 101).map { id => createServer(fromProps(createBrokerConfig(id, zkConnect))) } + brokers += createServer(fromProps(createBrokerConfig(100, zkConnect))) + assertEquals(100, TestUtils.waitUntilControllerElected(zkClient)) + + brokers += createServer(fromProps(createBrokerConfig(101, zkConnect))) + def leo() = brokers(1).replicaManager.getReplica(tp).get.logEndOffset.messageOffset adminZkClient.createOrUpdateTopicPartitionAssignmentPathInZK(tp.topic, Map(tp.partition -> Seq(101))) - producer = createNewProducer(getBrokerListStrFromServers(brokers), retries = 10, acks = -1) + producer = createProducer(getBrokerListStrFromServers(brokers), retries = 10, acks = -1) //1. Given a single message producer.send(new ProducerRecord(tp.topic, tp.partition, null, "IHeartLogs".getBytes)).get var fetcher = new TestFetcherThread(sender(brokers(0), brokers(1))) //Then epoch should be 0 and leo: 1 - var offset = fetcher.leaderOffsetsFor(Map(tp -> 0))(tp).endOffset() - assertEquals(1, offset) - assertEquals(leo(), offset) - + var epochEndOffset = fetcher.leaderOffsetsFor(Map(tp -> 0))(tp) + assertEquals(0, epochEndOffset.leaderEpoch) + assertEquals(1, epochEndOffset.endOffset) + assertEquals(1, leo()) //2. When broker is bounced brokers(1).shutdown() @@ -164,15 +167,23 @@ class LeaderEpochIntegrationTest extends ZooKeeperTestHarness with Logging { producer.send(new ProducerRecord(tp.topic, tp.partition, null, "IHeartLogs".getBytes)).get fetcher = new TestFetcherThread(sender(brokers(0), brokers(1))) - //Then epoch 0 should still be the start offset of epoch 1 - offset = fetcher.leaderOffsetsFor(Map(tp -> 0))(tp).endOffset() - assertEquals(1, offset) - - //Then epoch 2 should be the leo (NB: The leader epoch goes up in factors of 2 - This is because we have to first change leader to -1 and then change it again to the live replica) - assertEquals(2, fetcher.leaderOffsetsFor(Map(tp -> 2))(tp).endOffset()) - assertEquals(leo(), fetcher.leaderOffsetsFor(Map(tp -> 2))(tp).endOffset()) - + epochEndOffset = fetcher.leaderOffsetsFor(Map(tp -> 0))(tp) + assertEquals(1, epochEndOffset.endOffset) + assertEquals(0, epochEndOffset.leaderEpoch) + + //No data written in epoch 1 + epochEndOffset = fetcher.leaderOffsetsFor(Map(tp -> 1))(tp) + assertEquals(0, epochEndOffset.leaderEpoch) + assertEquals(1, epochEndOffset.endOffset) + + //Then epoch 2 should be the leo (NB: The leader epoch goes up in factors of 2 - + //This is because we have to first change leader to -1 and then change it again to the live replica) + //Note that the expected leader changes depend on the controller being on broker 100, which is not restarted + epochEndOffset = fetcher.leaderOffsetsFor(Map(tp -> 2))(tp) + assertEquals(2, epochEndOffset.leaderEpoch) + assertEquals(2, epochEndOffset.endOffset) + assertEquals(2, leo()) //3. When broker is bounced again brokers(1).shutdown() @@ -181,7 +192,6 @@ class LeaderEpochIntegrationTest extends ZooKeeperTestHarness with Logging { producer.send(new ProducerRecord(tp.topic, tp.partition, null, "IHeartLogs".getBytes)).get fetcher = new TestFetcherThread(sender(brokers(0), brokers(1))) - //Then Epoch 0 should still map to offset 1 assertEquals(1, fetcher.leaderOffsetsFor(Map(tp -> 0))(tp).endOffset()) @@ -252,7 +262,7 @@ class LeaderEpochIntegrationTest extends ZooKeeperTestHarness with Logging { private def sendFourMessagesToEachTopic() = { val testMessageList1 = List("test1", "test2", "test3", "test4") val testMessageList2 = List("test5", "test6", "test7", "test8") - val producer = TestUtils.createNewProducer(TestUtils.getBrokerListStrFromServers(brokers), retries = 5, keySerializer = new StringSerializer, valueSerializer = new StringSerializer) + val producer = TestUtils.createProducer(TestUtils.getBrokerListStrFromServers(brokers), retries = 5, keySerializer = new StringSerializer, valueSerializer = new StringSerializer) val records = testMessageList1.map(m => new ProducerRecord(topic1, m, m)) ++ testMessageList2.map(m => new ProducerRecord(topic2, m, m)) diff --git a/core/src/test/scala/unit/kafka/server/epoch/OffsetsForLeaderEpochTest.scala b/core/src/test/scala/unit/kafka/server/epoch/OffsetsForLeaderEpochTest.scala index 5c60c0017db38..0c317be3f6f2b 100644 --- a/core/src/test/scala/unit/kafka/server/epoch/OffsetsForLeaderEpochTest.scala +++ b/core/src/test/scala/unit/kafka/server/epoch/OffsetsForLeaderEpochTest.scala @@ -25,13 +25,13 @@ import kafka.utils.{MockTime, TestUtils} import org.apache.kafka.common.TopicPartition import org.apache.kafka.common.metrics.Metrics import org.apache.kafka.common.protocol.Errors +import org.apache.kafka.common.record.RecordVersion import org.apache.kafka.common.requests.EpochEndOffset import org.apache.kafka.common.requests.EpochEndOffset._ import org.easymock.EasyMock._ import org.junit.Assert._ import org.junit.Test - class OffsetsForLeaderEpochTest { private val config = TestUtils.createBrokerConfigs(1, TestUtils.MockZkConnect).map(KafkaConfig.fromProps).head private val time = new MockTime @@ -47,10 +47,12 @@ class OffsetsForLeaderEpochTest { //Stubs val mockLog = createNiceMock(classOf[kafka.log.Log]) - val mockCache = createNiceMock(classOf[kafka.server.epoch.LeaderEpochCache]) + val mockCache = createNiceMock(classOf[kafka.server.epoch.LeaderEpochFileCache]) val logManager = createNiceMock(classOf[kafka.log.LogManager]) expect(mockCache.endOffsetFor(epochRequested)).andReturn(epochAndOffset) expect(mockLog.leaderEpochCache).andReturn(mockCache).anyTimes() + expect(mockLog.supportsLeaderEpoch).andReturn(true).anyTimes() + expect(mockLog.recordVersion).andReturn(RecordVersion.V2).anyTimes() expect(logManager.liveLogDirs).andReturn(Array.empty[File]).anyTimes() replay(mockCache, mockLog, logManager) diff --git a/core/src/test/scala/unit/kafka/server/epoch/util/ReplicaFetcherMockBlockingSend.scala b/core/src/test/scala/unit/kafka/server/epoch/util/ReplicaFetcherMockBlockingSend.scala index b7c037ef86b39..fdb22b5f430c0 100644 --- a/core/src/test/scala/unit/kafka/server/epoch/util/ReplicaFetcherMockBlockingSend.scala +++ b/core/src/test/scala/unit/kafka/server/epoch/util/ReplicaFetcherMockBlockingSend.scala @@ -86,5 +86,7 @@ class ReplicaFetcherMockBlockingSend(offsets: java.util.Map[TopicPartition, Epoc true) } + override def initiateClose(): Unit = {} + override def close(): Unit = {} } diff --git a/core/src/test/scala/unit/kafka/tools/ConsoleConsumerTest.scala b/core/src/test/scala/unit/kafka/tools/ConsoleConsumerTest.scala index 1a32bf4207211..44ed5524102e6 100644 --- a/core/src/test/scala/unit/kafka/tools/ConsoleConsumerTest.scala +++ b/core/src/test/scala/unit/kafka/tools/ConsoleConsumerTest.scala @@ -20,7 +20,7 @@ package kafka.tools import java.io.{FileOutputStream, PrintStream} import kafka.common.MessageFormatter -import kafka.consumer.{BaseConsumer, BaseConsumerRecord, NewShinyConsumer} +import kafka.tools.ConsoleConsumer.ConsumerWrapper import kafka.utils.{Exit, TestUtils} import org.apache.kafka.clients.consumer.{ConsumerRecord, MockConsumer, OffsetResetStrategy} import org.apache.kafka.common.TopicPartition @@ -40,7 +40,7 @@ class ConsoleConsumerTest { } @Test - def shouldResetUnConsumedOffsetsBeforeExitForNewConsumer() { + def shouldResetUnConsumedOffsetsBeforeExit() { val topic = "test" val maxMessages: Int = 123 val totalMessages: Int = 700 @@ -50,7 +50,7 @@ class ConsoleConsumerTest { val tp1 = new TopicPartition(topic, 0) val tp2 = new TopicPartition(topic, 1) - val consumer = new NewShinyConsumer(Some(topic), None, None, None, mockConsumer) + val consumer = new ConsumerWrapper(Some(topic), None, None, None, mockConsumer) mockConsumer.rebalance(List(tp1, tp2).asJava) mockConsumer.updateBeginningOffsets(Map(tp1 -> startOffset, tp2 -> startOffset).asJava) @@ -80,11 +80,11 @@ class ConsoleConsumerTest { @Test def shouldLimitReadsToMaxMessageLimit() { //Mocks - val consumer = EasyMock.createNiceMock(classOf[BaseConsumer]) + val consumer = EasyMock.createNiceMock(classOf[ConsumerWrapper]) val formatter = EasyMock.createNiceMock(classOf[MessageFormatter]) //Stubs - val record = new BaseConsumerRecord(topic = "foo", partition = 1, offset = 1, key = Array[Byte](), value = Array[Byte]()) + val record = new ConsumerRecord("foo", 1, 1, Array[Byte](), Array[Byte]()) //Expectations val messageLimit: Int = 10 @@ -101,12 +101,12 @@ class ConsoleConsumerTest { @Test def shouldStopWhenOutputCheckErrorFails() { //Mocks - val consumer = EasyMock.createNiceMock(classOf[BaseConsumer]) + val consumer = EasyMock.createNiceMock(classOf[ConsumerWrapper]) val formatter = EasyMock.createNiceMock(classOf[MessageFormatter]) val printStream = EasyMock.createNiceMock(classOf[PrintStream]) //Stubs - val record = new BaseConsumerRecord(topic = "foo", partition = 1, offset = 1, key = Array[Byte](), value = Array[Byte]()) + val record = new ConsumerRecord("foo", 1, 1, Array[Byte](), Array[Byte]()) //Expectations EasyMock.expect(consumer.receive()).andReturn(record) @@ -126,25 +126,7 @@ class ConsoleConsumerTest { } @Test - def shouldParseValidOldConsumerValidConfig() { - //Given - val args: Array[String] = Array( - "--zookeeper", "localhost:2181", - "--topic", "test", - "--from-beginning") - - //When - val config = new ConsoleConsumer.ConsumerConfig(args) - - //Then - assertTrue(config.useOldConsumer) - assertEquals("localhost:2181", config.zkConnectionStr) - assertEquals("test", config.topicArg) - assertEquals(true, config.fromBeginning) - } - - @Test - def shouldParseValidNewConsumerValidConfig() { + def shouldParseValidConsumerValidConfig() { //Given val args: Array[String] = Array( "--bootstrap-server", "localhost:9092", @@ -155,14 +137,13 @@ class ConsoleConsumerTest { val config = new ConsoleConsumer.ConsumerConfig(args) //Then - assertFalse(config.useOldConsumer) assertEquals("localhost:9092", config.bootstrapServer) assertEquals("test", config.topicArg) assertEquals(true, config.fromBeginning) } @Test - def shouldParseValidNewSimpleConsumerValidConfigWithNumericOffset(): Unit = { + def shouldParseValidSimpleConsumerValidConfigWithNumericOffset(): Unit = { //Given val args: Array[String] = Array( "--bootstrap-server", "localhost:9092", @@ -174,7 +155,6 @@ class ConsoleConsumerTest { val config = new ConsoleConsumer.ConsumerConfig(args) //Then - assertFalse(config.useOldConsumer) assertEquals("localhost:9092", config.bootstrapServer) assertEquals("test", config.topicArg) assertEquals(0, config.partitionArg.get) @@ -203,37 +183,7 @@ class ConsoleConsumerTest { } @Test - def testDefaultConsumer() { - //Given - val args: Array[String] = Array( - "--bootstrap-server", "localhost:9092", - "--topic", "test", - "--from-beginning") - - //When - val config = new ConsoleConsumer.ConsumerConfig(args) - - //Then - assertFalse(config.useOldConsumer) - } - - @Test - def testNewConsumerRemovedOption() { - //Given - val args: Array[String] = Array( - "--bootstrap-server", "localhost:9092", - "--topic", "test", - "--from-beginning") - - //When - val config = new ConsoleConsumer.ConsumerConfig(args) - - //Then - assertFalse(config.useOldConsumer) - } - - @Test - def shouldParseValidNewSimpleConsumerValidConfigWithStringOffset() { + def shouldParseValidSimpleConsumerValidConfigWithStringOffset() { //Given val args: Array[String] = Array( "--bootstrap-server", "localhost:9092", @@ -246,7 +196,6 @@ class ConsoleConsumerTest { val config = new ConsoleConsumer.ConsumerConfig(args) //Then - assertFalse(config.useOldConsumer) assertEquals("localhost:9092", config.bootstrapServer) assertEquals("test", config.topicArg) assertEquals(0, config.partitionArg.get) @@ -256,67 +205,7 @@ class ConsoleConsumerTest { } @Test - def shouldParseValidOldConsumerConfigWithAutoOffsetResetSmallest() { - //Given - val args: Array[String] = Array( - "--zookeeper", "localhost:2181", - "--topic", "test", - "--consumer-property", "auto.offset.reset=smallest") - - //When - val config = new ConsoleConsumer.ConsumerConfig(args) - val consumerProperties = ConsoleConsumer.getOldConsumerProps(config) - - //Then - assertTrue(config.useOldConsumer) - assertEquals("localhost:2181", config.zkConnectionStr) - assertEquals("test", config.topicArg) - assertEquals(false, config.fromBeginning) - assertEquals("smallest", consumerProperties.getProperty(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG)) - } - - @Test - def shouldParseValidOldConsumerConfigWithAutoOffsetResetLargest() { - //Given - val args: Array[String] = Array( - "--zookeeper", "localhost:2181", - "--topic", "test", - "--consumer-property", "auto.offset.reset=largest") - - //When - val config = new ConsoleConsumer.ConsumerConfig(args) - val consumerProperties = ConsoleConsumer.getOldConsumerProps(config) - - //Then - assertTrue(config.useOldConsumer) - assertEquals("localhost:2181", config.zkConnectionStr) - assertEquals("test", config.topicArg) - assertEquals(false, config.fromBeginning) - assertEquals("largest", consumerProperties.getProperty(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG)) - } - - @Test - def shouldSetAutoResetToSmallestWhenFromBeginningConfigured() { - //Given - val args = Array( - "--zookeeper", "localhost:2181", - "--topic", "test", - "--from-beginning") - - //When - val config = new ConsoleConsumer.ConsumerConfig(args) - val consumerProperties = ConsoleConsumer.getOldConsumerProps(config) - - //Then - assertTrue(config.useOldConsumer) - assertEquals("localhost:2181", config.zkConnectionStr) - assertEquals("test", config.topicArg) - assertEquals(true, config.fromBeginning) - assertEquals("smallest", consumerProperties.getProperty(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG)) - } - - @Test - def shouldParseValidNewConsumerConfigWithAutoOffsetResetLatest() { + def shouldParseValidConsumerConfigWithAutoOffsetResetLatest() { //Given val args: Array[String] = Array( "--bootstrap-server", "localhost:9092", @@ -325,10 +214,9 @@ class ConsoleConsumerTest { //When val config = new ConsoleConsumer.ConsumerConfig(args) - val consumerProperties = ConsoleConsumer.getNewConsumerProps(config) + val consumerProperties = ConsoleConsumer.consumerProps(config) //Then - assertFalse(config.useOldConsumer) assertEquals("localhost:9092", config.bootstrapServer) assertEquals("test", config.topicArg) assertEquals(false, config.fromBeginning) @@ -336,7 +224,7 @@ class ConsoleConsumerTest { } @Test - def shouldParseValidNewConsumerConfigWithAutoOffsetResetEarliest() { + def shouldParseValidConsumerConfigWithAutoOffsetResetEarliest() { //Given val args: Array[String] = Array( "--bootstrap-server", "localhost:9092", @@ -345,10 +233,9 @@ class ConsoleConsumerTest { //When val config = new ConsoleConsumer.ConsumerConfig(args) - val consumerProperties = ConsoleConsumer.getNewConsumerProps(config) + val consumerProperties = ConsoleConsumer.consumerProps(config) //Then - assertFalse(config.useOldConsumer) assertEquals("localhost:9092", config.bootstrapServer) assertEquals("test", config.topicArg) assertEquals(false, config.fromBeginning) @@ -356,7 +243,7 @@ class ConsoleConsumerTest { } @Test - def shouldParseValidNewConsumerConfigWithAutoOffsetResetAndMatchingFromBeginning() { + def shouldParseValidConsumerConfigWithAutoOffsetResetAndMatchingFromBeginning() { //Given val args: Array[String] = Array( "--bootstrap-server", "localhost:9092", @@ -366,10 +253,9 @@ class ConsoleConsumerTest { //When val config = new ConsoleConsumer.ConsumerConfig(args) - val consumerProperties = ConsoleConsumer.getNewConsumerProps(config) + val consumerProperties = ConsoleConsumer.consumerProps(config) //Then - assertFalse(config.useOldConsumer) assertEquals("localhost:9092", config.bootstrapServer) assertEquals("test", config.topicArg) assertEquals(true, config.fromBeginning) @@ -377,7 +263,7 @@ class ConsoleConsumerTest { } @Test - def shouldParseValidNewConsumerConfigWithNoOffsetReset() { + def shouldParseValidConsumerConfigWithNoOffsetReset() { //Given val args: Array[String] = Array( "--bootstrap-server", "localhost:9092", @@ -385,10 +271,9 @@ class ConsoleConsumerTest { //When val config = new ConsoleConsumer.ConsumerConfig(args) - val consumerProperties = ConsoleConsumer.getNewConsumerProps(config) + val consumerProperties = ConsoleConsumer.consumerProps(config) //Then - assertFalse(config.useOldConsumer) assertEquals("localhost:9092", config.bootstrapServer) assertEquals("test", config.topicArg) assertEquals(false, config.fromBeginning) @@ -396,7 +281,7 @@ class ConsoleConsumerTest { } @Test(expected = classOf[IllegalArgumentException]) - def shouldExitOnInvalidConfigWithAutoOffsetResetAndConflictingFromBeginningNewConsumer() { + def shouldExitOnInvalidConfigWithAutoOffsetResetAndConflictingFromBeginning() { Exit.setExitProcedure((_, message) => throw new IllegalArgumentException(message.orNull)) //Given @@ -408,26 +293,7 @@ class ConsoleConsumerTest { try { val config = new ConsoleConsumer.ConsumerConfig(args) - ConsoleConsumer.getNewConsumerProps(config) - } finally { - Exit.resetExitProcedure() - } - } - - @Test(expected = classOf[IllegalArgumentException]) - def shouldExitOnInvalidConfigWithAutoOffsetResetAndConflictingFromBeginningOldConsumer() { - Exit.setExitProcedure((_, message) => throw new IllegalArgumentException(message.orNull)) - - //Given - val args: Array[String] = Array( - "--zookeeper", "localhost:2181", - "--topic", "test", - "--consumer-property", "auto.offset.reset=largest", - "--from-beginning") - - try { - val config = new ConsoleConsumer.ConsumerConfig(args) - ConsoleConsumer.getOldConsumerProps(config) + ConsoleConsumer.consumerProps(config) } finally { Exit.resetExitProcedure() } @@ -490,7 +356,7 @@ class ConsoleConsumerTest { ) var config = new ConsoleConsumer.ConsumerConfig(args) - var props = ConsoleConsumer.getNewConsumerProps(config) + var props = ConsoleConsumer.consumerProps(config) assertEquals("test-group", props.getProperty("group.id")) // different via --consumer-property and --consumer.config @@ -554,7 +420,7 @@ class ConsoleConsumerTest { ) config = new ConsoleConsumer.ConsumerConfig(args) - props = ConsoleConsumer.getNewConsumerProps(config) + props = ConsoleConsumer.consumerProps(config) assertEquals("group-from-arguments", props.getProperty("group.id")) Exit.resetExitProcedure() diff --git a/core/src/test/scala/unit/kafka/tools/ConsoleProducerTest.scala b/core/src/test/scala/unit/kafka/tools/ConsoleProducerTest.scala index 0e4b5b34c4296..86aae548e54ad 100644 --- a/core/src/test/scala/unit/kafka/tools/ConsoleProducerTest.scala +++ b/core/src/test/scala/unit/kafka/tools/ConsoleProducerTest.scala @@ -17,9 +17,12 @@ package kafka.tools +import java.util + import ConsoleProducer.LineMessageReader -import org.apache.kafka.clients.producer.KafkaProducer +import org.apache.kafka.clients.producer.ProducerConfig import org.junit.{Assert, Test} +import Assert.assertEquals class ConsoleProducerTest { @@ -40,12 +43,11 @@ class ConsoleProducerTest { ) @Test - def testValidConfigsNewProducer() { + def testValidConfigs() { 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 - val producer = new KafkaProducer(ConsoleProducer.producerProps(config)) - producer.close() + val producerConfig = new ProducerConfig(ConsoleProducer.producerProps(config)) + assertEquals(util.Arrays.asList("localhost:1001", "localhost:1002"), + producerConfig.getList(ProducerConfig.BOOTSTRAP_SERVERS_CONFIG)) } @Test diff --git a/core/src/test/scala/unit/kafka/tools/ConsumerPerformanceTest.scala b/core/src/test/scala/unit/kafka/tools/ConsumerPerformanceTest.scala index bc199f658abed..3beec10ae04e0 100644 --- a/core/src/test/scala/unit/kafka/tools/ConsumerPerformanceTest.scala +++ b/core/src/test/scala/unit/kafka/tools/ConsumerPerformanceTest.scala @@ -21,7 +21,7 @@ import java.io.ByteArrayOutputStream import java.text.SimpleDateFormat import joptsimple.OptionException -import org.junit.Assert.{assertEquals, assertFalse, assertTrue} +import org.junit.Assert.assertEquals import org.junit.Test class ConsumerPerformanceTest { @@ -31,23 +31,18 @@ class ConsumerPerformanceTest { @Test def testDetailedHeaderMatchBody(): Unit = { - testHeaderMatchContent(detailed = true, useOldConsumer = false, 2, - () => ConsumerPerformance.printNewConsumerProgress(1, 1024 * 1024, 0, 1, 0, 0, 1, dateFormat, 1L)) - testHeaderMatchContent(detailed = true, useOldConsumer = true, 4, - () => ConsumerPerformance.printOldConsumerProgress(1, 1024 * 1024, 0, 1, 0, 0, 1, - dateFormat)) + testHeaderMatchContent(detailed = true, 2, + () => ConsumerPerformance.printConsumerProgress(1, 1024 * 1024, 0, 1, 0, 0, 1, dateFormat, 1L)) } @Test def testNonDetailedHeaderMatchBody(): Unit = { - testHeaderMatchContent(detailed = false, useOldConsumer = false, 2, () => println(s"${dateFormat.format(System.currentTimeMillis)}, " + + testHeaderMatchContent(detailed = false, 2, () => println(s"${dateFormat.format(System.currentTimeMillis)}, " + s"${dateFormat.format(System.currentTimeMillis)}, 1.0, 1.0, 1, 1.0, 1, 1, 1.1, 1.1")) - testHeaderMatchContent(detailed = false, useOldConsumer = true, 4, () => println(s"${dateFormat.format(System.currentTimeMillis)}, " + - s"${dateFormat.format(System.currentTimeMillis)}, 1.0, 1.0, 1, 1.0")) } @Test - def testConfigUsingNewConsumer(): Unit = { + def testConfig(): Unit = { //Given val args: Array[String] = Array( "--broker-list", "localhost:9092", @@ -59,32 +54,13 @@ class ConsumerPerformanceTest { val config = new ConsumerPerformance.ConsumerPerfConfig(args) //Then - assertFalse(config.useOldConsumer) assertEquals("localhost:9092", config.options.valueOf(config.bootstrapServersOpt)) assertEquals("test", config.topic) assertEquals(10, config.numMessages) } - @Test - def testConfigUsingOldConsumer() { - //Given - val args: Array[String] = Array( - "--zookeeper", "localhost:2181", - "--topic", "test", - "--messages", "10") - - //When - val config = new ConsumerPerformance.ConsumerPerfConfig(args) - - //Then - assertTrue(config.useOldConsumer) - assertEquals("localhost:2181", config.options.valueOf(config.zkConnectOpt)) - assertEquals("test", config.topic) - assertEquals(10, config.numMessages) - } - @Test(expected = classOf[OptionException]) - def testConfigUsingNewConsumerUnrecognizedOption(): Unit = { + def testConfigWithUnrecognizedOption(): Unit = { //Given val args: Array[String] = Array( "--broker-list", "localhost:9092", @@ -97,9 +73,9 @@ class ConsumerPerformanceTest { new ConsumerPerformance.ConsumerPerfConfig(args) } - private def testHeaderMatchContent(detailed: Boolean, useOldConsumer: Boolean, expectedOutputLineCount: Int, fun: () => Unit): Unit = { + private def testHeaderMatchContent(detailed: Boolean, expectedOutputLineCount: Int, fun: () => Unit): Unit = { Console.withOut(outContent) { - ConsumerPerformance.printHeader(detailed, useOldConsumer) + ConsumerPerformance.printHeader(detailed) fun() val contents = outContent.toString.split("\n") diff --git a/core/src/test/scala/unit/kafka/utils/CoreUtilsTest.scala b/core/src/test/scala/unit/kafka/utils/CoreUtilsTest.scala index 7c416a26917ef..93578c616e682 100755 --- a/core/src/test/scala/unit/kafka/utils/CoreUtilsTest.scala +++ b/core/src/test/scala/unit/kafka/utils/CoreUtilsTest.scala @@ -26,8 +26,8 @@ import java.util.regex.Pattern import org.scalatest.junit.JUnitSuite import org.junit.Assert._ -import kafka.common.KafkaException import kafka.utils.CoreUtils.inLock +import org.apache.kafka.common.KafkaException import org.junit.Test import org.apache.kafka.common.utils.Utils import org.slf4j.event.Level diff --git a/core/src/test/scala/unit/kafka/utils/TestUtils.scala b/core/src/test/scala/unit/kafka/utils/TestUtils.scala index f50ef3a8a35d7..c283cad6e1c1f 100755 --- a/core/src/test/scala/unit/kafka/utils/TestUtils.scala +++ b/core/src/test/scala/unit/kafka/utils/TestUtils.scala @@ -22,15 +22,14 @@ import java.nio._ import java.nio.channels._ import java.nio.charset.{Charset, StandardCharsets} import java.security.cert.X509Certificate +import java.time.Duration import java.util.{Collections, Properties} -import java.util.concurrent.{Callable, Executors, TimeUnit} +import java.util.concurrent.{Callable, ExecutionException, Executors, TimeUnit} import javax.net.ssl.X509TrustManager import kafka.api._ import kafka.cluster.{Broker, EndPoint} -import kafka.consumer.{ConsumerConfig, ConsumerTimeoutException, KafkaStream} import kafka.log._ -import kafka.message._ import kafka.security.auth.{Acl, Authorizer, Resource} import kafka.server._ import kafka.server.checkpoints.OffsetCheckpointFile @@ -41,7 +40,7 @@ import org.apache.kafka.clients.CommonClientConfigs import org.apache.kafka.clients.admin.{AdminClient, AlterConfigsResult, Config, ConfigEntry} import org.apache.kafka.clients.consumer.{ConsumerRecord, KafkaConsumer, OffsetAndMetadata, RangeAssignor} import org.apache.kafka.clients.producer.{KafkaProducer, ProducerConfig, ProducerRecord} -import org.apache.kafka.common.TopicPartition +import org.apache.kafka.common.{KafkaFuture, TopicPartition} import org.apache.kafka.common.config.ConfigResource import org.apache.kafka.common.header.Header import org.apache.kafka.common.internals.Topic @@ -70,6 +69,10 @@ object TestUtils extends Logging { /* 0 gives a random port; you can then retrieve the assigned port from the Socket object. */ val RandomPort = 0 + /* Incorrect broker port which can used by kafka clients in tests. This port should not be used + by any other service and hence we use a reserved port. */ + val IncorrectBrokerPort = 225 + /** 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. */ @@ -376,10 +379,11 @@ object TestUtils extends Logging { producerId: Long = RecordBatch.NO_PRODUCER_ID, producerEpoch: Short = RecordBatch.NO_PRODUCER_EPOCH, sequence: Int = RecordBatch.NO_SEQUENCE, - baseOffset: Long = 0L): MemoryRecords = { + baseOffset: Long = 0L, + partitionLeaderEpoch: Int = RecordBatch.NO_PARTITION_LEADER_EPOCH): MemoryRecords = { val buf = ByteBuffer.allocate(DefaultRecordBatch.sizeInBytes(records.asJava)) val builder = MemoryRecords.builder(buf, magicValue, codec, TimestampType.CREATE_TIME, baseOffset, - System.currentTimeMillis, producerId, producerEpoch, sequence) + System.currentTimeMillis, producerId, producerEpoch, sequence, false, partitionLeaderEpoch) records.foreach(builder.append) builder.build() } @@ -526,19 +530,19 @@ object TestUtils extends Logging { /** * Create a (new) producer with a few pre-configured properties. */ - def createNewProducer[K, V](brokerList: String, - acks: Int = -1, - maxBlockMs: Long = 60 * 1000L, - bufferSize: Long = 1024L * 1024L, - retries: Int = 0, - lingerMs: Long = 0, - requestTimeoutMs: Long = 30 * 1000L, - securityProtocol: SecurityProtocol = SecurityProtocol.PLAINTEXT, - trustStoreFile: Option[File] = None, - saslProperties: Option[Properties] = None, - keySerializer: Serializer[K] = new ByteArraySerializer, - valueSerializer: Serializer[V] = new ByteArraySerializer, - props: Option[Properties] = None): KafkaProducer[K, V] = { + def createProducer[K, V](brokerList: String, + acks: Int = -1, + maxBlockMs: Long = 60 * 1000L, + bufferSize: Long = 1024L * 1024L, + retries: Int = 0, + lingerMs: Long = 0, + requestTimeoutMs: Long = 30 * 1000L, + securityProtocol: SecurityProtocol = SecurityProtocol.PLAINTEXT, + trustStoreFile: Option[File] = None, + saslProperties: Option[Properties] = None, + keySerializer: Serializer[K] = new ByteArraySerializer, + valueSerializer: Serializer[V] = new ByteArraySerializer, + props: Option[Properties] = None): KafkaProducer[K, V] = { val producerProps = props.getOrElse(new Properties) producerProps.put(ProducerConfig.BOOTSTRAP_SERVERS_CONFIG, brokerList) @@ -588,20 +592,20 @@ object TestUtils extends Logging { securityConfigs(Mode.CLIENT, securityProtocol, trustStoreFile, "admin-client", SslCertificateCn, saslProperties) /** - * Create a new consumer with a few pre-configured properties. + * Create a consumer with a few pre-configured properties. */ - def createNewConsumer[K, V](brokerList: String, - groupId: String = "group", - autoOffsetReset: String = "earliest", - partitionFetchSize: Long = 4096L, - partitionAssignmentStrategy: String = classOf[RangeAssignor].getName, - sessionTimeout: Int = 30000, - securityProtocol: SecurityProtocol, - trustStoreFile: Option[File] = None, - saslProperties: Option[Properties] = None, - keyDeserializer: Deserializer[K] = new ByteArrayDeserializer, - valueDeserializer: Deserializer[V] =new ByteArrayDeserializer, - props: Option[Properties] = None) : KafkaConsumer[K, V] = { + def createConsumer[K, V](brokerList: String, + groupId: String = "group", + autoOffsetReset: String = "earliest", + partitionFetchSize: Long = 4096L, + partitionAssignmentStrategy: String = classOf[RangeAssignor].getName, + sessionTimeout: Int = 30000, + securityProtocol: SecurityProtocol, + trustStoreFile: Option[File] = None, + saslProperties: Option[Properties] = None, + keyDeserializer: Deserializer[K] = new ByteArrayDeserializer, + valueDeserializer: Deserializer[V] = new ByteArrayDeserializer, + props: Option[Properties] = None): KafkaConsumer[K, V] = { import org.apache.kafka.clients.consumer.ConsumerConfig val consumerProps = props.getOrElse(new Properties()) @@ -647,25 +651,6 @@ object TestUtils extends Logging { props } - @deprecated("This method has been deprecated and will be removed in a future release.", "0.11.0.0") - def updateConsumerOffset(config : ConsumerConfig, path : String, offset : Long) = { - val zkUtils = ZkUtils(config.zkConnect, config.zkSessionTimeoutMs, config.zkConnectionTimeoutMs, false) - zkUtils.updatePersistentPath(path, offset.toString) - zkUtils.close() - - } - - def getMessageIterator(iter: Iterator[MessageAndOffset]): Iterator[Message] = { - new IteratorTemplate[Message] { - override def makeNext(): Message = { - if (iter.hasNext) - iter.next.message - else - allDone() - } - } - } - def createBrokersInZk(zkClient: KafkaZkClient, ids: Seq[Int]): Seq[Broker] = createBrokersInZk(ids.map(kafka.admin.BrokerMetadata(_, None)), zkClient) @@ -832,14 +817,6 @@ object TestUtils extends Logging { server.replicaManager.getPartition(new TopicPartition(topic, partitionId)).exists(_.leaderReplicaIfLocal.isDefined) } - def createRequestByteBuffer(request: RequestOrResponse): ByteBuffer = { - val byteBuffer = ByteBuffer.allocate(request.sizeInBytes + 2) - byteBuffer.putShort(request.requestId.get) - request.writeTo(byteBuffer) - byteBuffer.rewind() - byteBuffer - } - /** * Wait until all brokers know about each other. * @@ -957,7 +934,7 @@ object TestUtils extends Logging { defaultConfig: LogConfig = LogConfig(), cleanerConfig: CleanerConfig = CleanerConfig(enableCleaner = false), time: MockTime = new MockTime()): LogManager = { - new LogManager(logDirs = logDirs, + new LogManager(logDirs = logDirs.map(_.getAbsoluteFile), initialOfflineDirs = Array.empty[File], topicConfigs = Map(), initialDefaultConfig = defaultConfig, @@ -981,7 +958,7 @@ object TestUtils extends Logging { compressionType: CompressionType = CompressionType.NONE): Unit = { val props = new Properties() props.put(ProducerConfig.COMPRESSION_TYPE_CONFIG, compressionType.name) - val producer = createNewProducer(TestUtils.getBrokerListStrFromServers(servers), retries = 5, acks = acks) + val producer = createProducer(TestUtils.getBrokerListStrFromServers(servers), retries = 5, acks = acks) try { val futures = records.map(producer.send) futures.foreach(_.get) @@ -1005,7 +982,7 @@ object TestUtils extends Logging { } def produceMessage(servers: Seq[KafkaServer], topic: String, message: String) { - val producer = createNewProducer( + val producer = createProducer( TestUtils.getBrokerListStrFromServers(servers), retries = 5 ) @@ -1013,48 +990,6 @@ object TestUtils extends Logging { 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. - */ - @deprecated("This method has been deprecated and will be removed in a future release.", "0.11.0.0") - def getMessages(topicMessageStreams: Map[String, List[KafkaStream[String, String]]], - nMessagesPerThread: Int = -1): List[String] = { - - var messages: List[String] = Nil - val shouldGetAllMessages = nMessagesPerThread < 0 - for (messageStreams <- topicMessageStreams.values) { - 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: KafkaZkClient, topic: String, numPartitions: Int, servers: Seq[KafkaServer]) { val topicPartitions = (0 until numPartitions).map(new TopicPartition(topic, _)) // wait until admin path for delete topic is deleted, signaling completion of topic deletion @@ -1258,10 +1193,11 @@ object TestUtils extends Logging { def consumeTopicRecords[K, V](servers: Seq[KafkaServer], topic: String, numMessages: Int, + groupId: String = "group", securityProtocol: SecurityProtocol = SecurityProtocol.PLAINTEXT, trustStoreFile: Option[File] = None, waitTime: Long = JTestUtils.DEFAULT_MAX_WAIT_MS): Seq[ConsumerRecord[Array[Byte], Array[Byte]]] = { - val consumer = createNewConsumer(TestUtils.getBrokerListStrFromServers(servers, securityProtocol), + val consumer = createConsumer(TestUtils.getBrokerListStrFromServers(servers, securityProtocol), groupId = groupId, securityProtocol = securityProtocol, trustStoreFile = trustStoreFile) try { consumer.subscribe(Collections.singleton(topic)) @@ -1273,7 +1209,7 @@ object TestUtils extends Logging { waitTime: Long = JTestUtils.DEFAULT_MAX_WAIT_MS): Seq[ConsumerRecord[K, V]] = { val records = new ArrayBuffer[ConsumerRecord[K, V]]() waitUntilTrue(() => { - records ++= consumer.poll(50).asScala + records ++= consumer.poll(Duration.ofMillis(50)).asScala records.size >= numMessages }, s"Consumed ${records.size} records until timeout instead of the expected $numMessages records", waitTime) assertEquals("Consumed more records than expected", numMessages, records.size) @@ -1288,11 +1224,11 @@ object TestUtils extends Logging { * * @return All the records consumed by the consumer within the specified duration. */ - def consumeRecordsFor[K, V](consumer: KafkaConsumer[K, V], duration: Long): Seq[ConsumerRecord[K, V]] = { + def consumeRecordsFor[K, V](consumer: KafkaConsumer[K, V], duration: Long = JTestUtils.DEFAULT_MAX_WAIT_MS): Seq[ConsumerRecord[K, V]] = { val startTime = System.currentTimeMillis() val records = new ArrayBuffer[ConsumerRecord[K, V]]() waitUntilTrue(() => { - records ++= consumer.poll(50).asScala + records ++= consumer.poll(Duration.ofMillis(50)).asScala System.currentTimeMillis() - startTime > duration }, s"The timeout $duration was greater than the maximum wait time.") records @@ -1306,14 +1242,14 @@ object TestUtils extends Logging { props.put(ProducerConfig.ENABLE_IDEMPOTENCE_CONFIG, "true") props.put(ProducerConfig.BATCH_SIZE_CONFIG, batchSize.toString) props.put(ProducerConfig.TRANSACTION_TIMEOUT_CONFIG, transactionTimeoutMs.toString) - TestUtils.createNewProducer(TestUtils.getBrokerListStrFromServers(servers), retries = Integer.MAX_VALUE, acks = -1, props = Some(props)) + TestUtils.createProducer(TestUtils.getBrokerListStrFromServers(servers), retries = Integer.MAX_VALUE, acks = -1, props = Some(props)) } // Seeds the given topic with records with keys and values in the range [0..numRecords) def seedTopicWithNumberedRecords(topic: String, numRecords: Int, servers: Seq[KafkaServer]): Unit = { val props = new Properties() props.put(ProducerConfig.ENABLE_IDEMPOTENCE_CONFIG, "true") - val producer = TestUtils.createNewProducer(TestUtils.getBrokerListStrFromServers(servers), + val producer = TestUtils.createProducer(TestUtils.getBrokerListStrFromServers(servers), retries = Integer.MAX_VALUE, acks = -1, props = Some(props)) try { for (i <- 0 until numRecords) { @@ -1375,14 +1311,14 @@ object TestUtils extends Logging { def pollUntilAtLeastNumRecords(consumer: KafkaConsumer[Array[Byte], Array[Byte]], numRecords: Int): Seq[ConsumerRecord[Array[Byte], Array[Byte]]] = { val records = new ArrayBuffer[ConsumerRecord[Array[Byte], Array[Byte]]]() TestUtils.waitUntilTrue(() => { - records ++= consumer.poll(50).asScala + records ++= consumer.poll(Duration.ofMillis(50)).asScala records.size >= numRecords }, s"Consumed ${records.size} records until timeout, but expected $numRecords records.") records } def resetToCommittedPositions(consumer: KafkaConsumer[Array[Byte], Array[Byte]]) = { - consumer.assignment.asScala.foreach { case(topicPartition) => + consumer.assignment.asScala.foreach { topicPartition => val offset = consumer.committed(topicPartition) if (offset != null) consumer.seek(topicPartition, offset.offset) @@ -1440,4 +1376,22 @@ object TestUtils extends Logging { (out.toString, err.toString) } + def assertFutureExceptionTypeEquals(future: KafkaFuture[_], clazz: Class[_ <: Throwable]): Unit = { + try { + future.get() + fail("Expected CompletableFuture.get to return an exception") + } catch { + case e: ExecutionException => + val cause = e.getCause() + assertTrue("Expected an exception of type " + clazz.getName + "; got type " + + cause.getClass().getName, clazz.isInstance(cause)) + } + } + + def totalMetricValue(server: KafkaServer, metricName: String): Long = { + val allMetrics = server.metrics.metrics + val total = allMetrics.values().asScala.filter(_.metricName().name() == metricName) + .foldLeft(0.0)((total, metric) => total + metric.metricValue.asInstanceOf[Double]) + total.toLong + } } diff --git a/core/src/test/scala/unit/kafka/consumer/TopicFilterTest.scala b/core/src/test/scala/unit/kafka/utils/TopicFilterTest.scala similarity index 53% rename from core/src/test/scala/unit/kafka/consumer/TopicFilterTest.scala rename to core/src/test/scala/unit/kafka/utils/TopicFilterTest.scala index ca8a0d627179d..19cd1b65f8e98 100644 --- a/core/src/test/scala/unit/kafka/consumer/TopicFilterTest.scala +++ b/core/src/test/scala/unit/kafka/utils/TopicFilterTest.scala @@ -15,15 +15,13 @@ * limitations under the License. */ -package kafka.consumer - +package kafka.utils import org.apache.kafka.common.internals.Topic import org.junit.Assert._ -import org.scalatest.junit.JUnitSuite import org.junit.Test +import org.scalatest.junit.JUnitSuite -@deprecated("This test has been deprecated and will be removed in a future release.", "0.11.0.0") class TopicFilterTest extends JUnitSuite { @Test @@ -49,37 +47,4 @@ class TopicFilterTest extends JUnitSuite { assertFalse(topicFilter4.isTopicAllowed("test-bad", excludeInternalTopics = true)) } - @Test - def testBlacklists() { - val topicFilter1 = Blacklist("black1") - assertTrue(topicFilter1.isTopicAllowed("white2", excludeInternalTopics = true)) - assertTrue(topicFilter1.isTopicAllowed("white2", excludeInternalTopics = false)) - assertFalse(topicFilter1.isTopicAllowed("black1", excludeInternalTopics = true)) - assertFalse(topicFilter1.isTopicAllowed("black1", excludeInternalTopics = false)) - - assertFalse(topicFilter1.isTopicAllowed(Topic.GROUP_METADATA_TOPIC_NAME, excludeInternalTopics = true)) - assertTrue(topicFilter1.isTopicAllowed(Topic.GROUP_METADATA_TOPIC_NAME, excludeInternalTopics = false)) - } - - @Test - def testWildcardTopicCountGetTopicCountMapEscapeJson() { - def getTopicCountMapKey(regex: String): String = { - val topicCount = new WildcardTopicCount(null, "consumerId", new Whitelist(regex), 1, true) - topicCount.getTopicCountMap.head._1 - } - //lets make sure that the JSON strings are escaping as we expect - //if they are not then when they get saved to ZooKeeper and read back out they will be broken on parse - assertEquals("-\\\"-", getTopicCountMapKey("-\"-")) - assertEquals("-\\\\-", getTopicCountMapKey("-\\-")) - assertEquals("-\\/-", getTopicCountMapKey("-/-")) - assertEquals("-\\\\b-", getTopicCountMapKey("-\\b-")) - assertEquals("-\\\\f-", getTopicCountMapKey("-\\f-")) - assertEquals("-\\\\n-", getTopicCountMapKey("-\\n-")) - assertEquals("-\\\\r-", getTopicCountMapKey("-\\r-")) - assertEquals("-\\\\t-", getTopicCountMapKey("-\\t-")) - assertEquals("-\\\\u0000-", getTopicCountMapKey("-\\u0000-")) - assertEquals("-\\\\u001f-", getTopicCountMapKey("-\\u001f-")) - assertEquals("-\\\\u007f-", getTopicCountMapKey("-\\u007f-")) - assertEquals("-\\\\u009f-", getTopicCountMapKey("-\\u009f-")) - } } diff --git a/core/src/test/scala/unit/kafka/zk/AdminZkClientTest.scala b/core/src/test/scala/unit/kafka/zk/AdminZkClientTest.scala index fe5fbff55d05d..39745e5e608dd 100644 --- a/core/src/test/scala/unit/kafka/zk/AdminZkClientTest.scala +++ b/core/src/test/scala/unit/kafka/zk/AdminZkClientTest.scala @@ -28,8 +28,10 @@ import kafka.utils.TestUtils._ import kafka.utils.{Logging, TestUtils} import kafka.zk.{AdminZkClient, KafkaZkClient, ZooKeeperTestHarness} import org.apache.kafka.common.TopicPartition +import org.apache.kafka.common.config.TopicConfig import org.apache.kafka.common.errors.{InvalidReplicaAssignmentException, InvalidTopicException, TopicExistsException} import org.apache.kafka.common.metrics.Quota +import org.apache.kafka.test.{TestUtils => JTestUtils} import org.easymock.EasyMock import org.junit.Assert._ import org.junit.{After, Test} @@ -132,7 +134,7 @@ class AdminZkClientTest extends ZooKeeperTestHarness with Logging with RackAware } @Test - def testConcurrentTopicCreation() { + def testMockedConcurrentTopicCreation() { val topic = "test.topic" // simulate the ZK interactions that can happen when a topic is concurrently created by multiple processes @@ -147,6 +149,28 @@ class AdminZkClientTest extends ZooKeeperTestHarness with Logging with RackAware } } + @Test + def testConcurrentTopicCreation() { + val topic = "test-concurrent-topic-creation" + TestUtils.createBrokersInZk(zkClient, List(0, 1, 2, 3, 4)) + val props = new Properties + props.setProperty(TopicConfig.MIN_IN_SYNC_REPLICAS_CONFIG, "2") + def createTopic(): Unit = { + try adminZkClient.createTopic(topic, 3, 1, props) + catch { case _: TopicExistsException => () } + val (_, partitionAssignment) = zkClient.getPartitionAssignmentForTopics(Set(topic)).head + assertEquals(3, partitionAssignment.size) + partitionAssignment.foreach { case (partition, replicas) => + assertEquals(s"Unexpected replication factor for $partition", 1, replicas.size) + } + val savedProps = zkClient.getEntityConfigs(ConfigType.Topic, topic) + assertEquals(props, savedProps) + } + + TestUtils.assertConcurrent("Concurrent topic creation failed", Seq(createTopic, createTopic), + JTestUtils.DEFAULT_MAX_WAIT_MS.toInt) + } + /** * This test creates a topic with a few config overrides and checks that the configs are applied to the new topic * then changes the config and checks that the new values take effect. diff --git a/core/src/test/scala/unit/kafka/zk/KafkaZkClientTest.scala b/core/src/test/scala/unit/kafka/zk/KafkaZkClientTest.scala index cfaf731768019..77da8bdf57513 100644 --- a/core/src/test/scala/unit/kafka/zk/KafkaZkClientTest.scala +++ b/core/src/test/scala/unit/kafka/zk/KafkaZkClientTest.scala @@ -34,11 +34,11 @@ import org.apache.kafka.common.utils.{SecurityUtils, Time} import org.apache.zookeeper.KeeperException.{Code, NoNodeException, NodeExistsException} import org.junit.Assert._ import org.junit.{After, Before, Test} + import scala.collection.JavaConverters._ import scala.collection.mutable.ArrayBuffer import scala.collection.{Seq, mutable} import scala.util.Random - import kafka.controller.LeaderIsrAndControllerEpoch import kafka.zk.KafkaZkClient.UpdateLeaderAndIsrResult import kafka.zookeeper._ @@ -176,7 +176,7 @@ class KafkaZkClientTest extends ZooKeeperTestHarness { // test with non-existing path val (data0, version0) = zkClient.getDataAndVersion(path) assertTrue(data0.isEmpty) - assertEquals(-1, version0) + assertEquals(ZkVersion.UnknownVersion, version0) // create a test path zkClient.createRecursive(path) @@ -200,7 +200,7 @@ class KafkaZkClientTest extends ZooKeeperTestHarness { // test with non-existing path var statusAndVersion = zkClient.conditionalUpdatePath(path, "version0".getBytes(UTF_8), 0) assertFalse(statusAndVersion._1) - assertEquals(-1, statusAndVersion._2) + assertEquals(ZkVersion.UnknownVersion, statusAndVersion._2) // create path zkClient.createRecursive(path) @@ -213,7 +213,7 @@ class KafkaZkClientTest extends ZooKeeperTestHarness { // test with invalid expected version statusAndVersion = zkClient.conditionalUpdatePath(path, "version2".getBytes(UTF_8), 2) assertFalse(statusAndVersion._1) - assertEquals(-1, statusAndVersion._2) + assertEquals(ZkVersion.UnknownVersion, statusAndVersion._2) } @Test @@ -426,10 +426,9 @@ class KafkaZkClientTest extends ZooKeeperTestHarness { @Test def testAclManagementMethods() { - ZkAclStore.stores.foreach(store => { assertFalse(zkClient.pathExists(store.aclPath)) - assertFalse(zkClient.pathExists(store.aclChangePath)) + assertFalse(zkClient.pathExists(store.changeStore.aclChangePath)) ResourceType.values.foreach(resource => assertFalse(zkClient.pathExists(store.path(resource)))) }) @@ -438,16 +437,16 @@ class KafkaZkClientTest extends ZooKeeperTestHarness { ZkAclStore.stores.foreach(store => { assertTrue(zkClient.pathExists(store.aclPath)) - assertTrue(zkClient.pathExists(store.aclChangePath)) + assertTrue(zkClient.pathExists(store.changeStore.aclChangePath)) ResourceType.values.foreach(resource => assertTrue(zkClient.pathExists(store.path(resource)))) - val resource1 = new Resource(Topic, UUID.randomUUID().toString, store.nameType) - val resource2 = new Resource(Topic, UUID.randomUUID().toString, store.nameType) + val resource1 = new Resource(Topic, UUID.randomUUID().toString, store.patternType) + val resource2 = new Resource(Topic, UUID.randomUUID().toString, store.patternType) // try getting acls for non-existing resource var versionedAcls = zkClient.getVersionedAclsForResource(resource1) assertTrue(versionedAcls.acls.isEmpty) - assertEquals(-1, versionedAcls.zkVersion) + assertEquals(ZkVersion.UnknownVersion, versionedAcls.zkVersion) assertFalse(zkClient.resourceExists(resource1)) @@ -455,9 +454,15 @@ class KafkaZkClientTest extends ZooKeeperTestHarness { 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) + // Conditional set should fail if path not created + assertFalse(zkClient.conditionalSetAclsForResource(resource1, Set(acl1, acl3), 0)._1) + //create acls for resources - zkClient.conditionalSetOrCreateAclsForResource(resource1, Set(acl1, acl2), 0) - zkClient.conditionalSetOrCreateAclsForResource(resource2, Set(acl1, acl3), 0) + assertTrue(zkClient.createAclsForResourceIfNotExists(resource1, Set(acl1, acl2))._1) + assertTrue(zkClient.createAclsForResourceIfNotExists(resource2, Set(acl1, acl3))._1) + + // Create should fail if path already exists + assertFalse(zkClient.createAclsForResourceIfNotExists(resource2, Set(acl1, acl3))._1) versionedAcls = zkClient.getVersionedAclsForResource(resource1) assertEquals(Set(acl1, acl2), versionedAcls.acls) @@ -465,17 +470,17 @@ class KafkaZkClientTest extends ZooKeeperTestHarness { assertTrue(zkClient.resourceExists(resource1)) //update acls for resource - zkClient.conditionalSetOrCreateAclsForResource(resource1, Set(acl1, acl3), 0) + assertTrue(zkClient.conditionalSetAclsForResource(resource1, Set(acl1, acl3), 0)._1) versionedAcls = zkClient.getVersionedAclsForResource(resource1) assertEquals(Set(acl1, acl3), versionedAcls.acls) assertEquals(1, versionedAcls.zkVersion) //get resource Types - assertTrue(ResourceType.values.map( rt => rt.name).toSet == zkClient.getResourceTypes(store.nameType).toSet) + assertTrue(ResourceType.values.map( rt => rt.name).toSet == zkClient.getResourceTypes(store.patternType).toSet) //get resource name - val resourceNames = zkClient.getResourceNames(store.nameType, Topic) + val resourceNames = zkClient.getResourceNames(store.patternType, Topic) assertEquals(2, resourceNames.size) assertTrue(Set(resource1.name,resource2.name) == resourceNames.toSet) @@ -488,14 +493,13 @@ class KafkaZkClientTest extends ZooKeeperTestHarness { //delete with valid expected zk version assertTrue(zkClient.conditionalDelete(resource2, 0)) + zkClient.createAclChangeNotification(Resource(Group, "resource1", store.patternType)) + zkClient.createAclChangeNotification(Resource(Topic, "resource2", store.patternType)) - zkClient.createAclChangeNotification(Resource(Group, "resource1", store.nameType)) - zkClient.createAclChangeNotification(Resource(Topic, "resource2", store.nameType)) - - assertEquals(2, zkClient.getChildren(store.aclChangePath).size) + assertEquals(2, zkClient.getChildren(store.changeStore.aclChangePath).size) zkClient.deleteAclChangeNotifications() - assertTrue(zkClient.getChildren(store.aclChangePath).isEmpty) + assertTrue(zkClient.getChildren(store.changeStore.aclChangePath).isEmpty) }) } diff --git a/core/src/test/scala/unit/kafka/zk/ZKEphemeralTest.scala b/core/src/test/scala/unit/kafka/zk/ZKEphemeralTest.scala deleted file mode 100644 index 06ea963077214..0000000000000 --- a/core/src/test/scala/unit/kafka/zk/ZKEphemeralTest.scala +++ /dev/null @@ -1,177 +0,0 @@ -/** - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. -*/ - -package kafka.zk - -import java.lang.Iterable -import javax.security.auth.login.Configuration - -import scala.collection.JavaConverters._ -import kafka.consumer.ConsumerConfig -import kafka.utils.{CoreUtils, TestUtils, ZKCheckedEphemeral, ZkUtils} -import org.apache.kafka.common.security.JaasUtils -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.ZkNodeExistsException -import org.junit.{After, Assert, Before, Test} -import org.junit.runners.Parameterized -import org.junit.runners.Parameterized.Parameters -import org.junit.runner.RunWith - -object ZKEphemeralTest { - - @Parameters - def enableSecurityOptions: Iterable[Array[java.lang.Boolean]] = - Seq[Array[java.lang.Boolean]](Array(true), Array(false)).asJava - -} - -@RunWith(value = classOf[Parameterized]) -class ZKEphemeralTest(val secure: Boolean) extends ZooKeeperTestHarness { - val jaasFile = kafka.utils.JaasTestUtils.writeJaasContextsToFile(kafka.utils.JaasTestUtils.zkSections) - val authProvider = "zookeeper.authProvider.1" - var zkSessionTimeoutMs = 1000 - var zkUtils: ZkUtils = null - - @Before - override def setUp() { - if (secure) { - System.setProperty(JaasUtils.JAVA_LOGIN_CONFIG_PARAM, jaasFile.getAbsolutePath) - Configuration.setConfiguration(null) - System.setProperty(authProvider, "org.apache.zookeeper.server.auth.SASLAuthenticationProvider") - if (!JaasUtils.isZkSecurityEnabled) - fail("Secure access not enabled") - } - super.setUp - zkUtils = ZkUtils(zkConnect, zkSessionTimeout, zkConnectionTimeout, zkAclsEnabled.getOrElse(JaasUtils.isZkSecurityEnabled)) - } - - @After - override def tearDown() { - if (zkUtils != null) - CoreUtils.swallow(zkUtils.close(), this) - super.tearDown - System.clearProperty(JaasUtils.JAVA_LOGIN_CONFIG_PARAM) - System.clearProperty(authProvider) - Configuration.setConfiguration(null) - } - - @Test - def testEphemeralNodeCleanup(): Unit = { - val config = new ConsumerConfig(TestUtils.createConsumerProperties(zkConnect, "test", "1")) - var zkUtils = ZkUtils(zkConnect, zkSessionTimeoutMs, config.zkConnectionTimeoutMs, JaasUtils.isZkSecurityEnabled()) - - try { - zkUtils.createEphemeralPathExpectConflict("/tmp/zktest", "node created") - } catch { - case _: Exception => - } - - var testData: String = null - testData = zkUtils.readData("/tmp/zktest")._1 - Assert.assertNotNull(testData) - zkUtils.close - zkUtils = ZkUtils(zkConnect, zkSessionTimeoutMs, config.zkConnectionTimeoutMs, JaasUtils.isZkSecurityEnabled()) - val nodeExists = zkUtils.pathExists("/tmp/zktest") - Assert.assertFalse(nodeExists) - zkUtils.close() - } - - /***** - ***** Tests for ZkWatchedEphemeral - *****/ - - /** - * Tests basic creation - */ - @Test - def testZkWatchedEphemeral(): Unit = { - testCreation("/zwe-test") - testCreation("/zwe-test-parent/zwe-test") - } - - private def testCreation(path: String) { - val zk = zkUtils.zkConnection.getZookeeper - val zwe = new ZKCheckedEphemeral(path, "", zk, JaasUtils.isZkSecurityEnabled()) - var created = false - - 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(path), - s"Znode $path wasn't created") - } - - /** - * Tests that it fails in the presence of an overlapping - * session. - */ - @Test - def testOverlappingSessions(): Unit = { - val path = "/zwe-test" - val zk1 = zkUtils.zkConnection.getZookeeper - - //Creates a second session - val (zkClient2, zkConnection2) = ZkUtils.createZkClientAndConnection(zkConnect, zkSessionTimeoutMs, zkConnectionTimeout) - val zk2 = zkConnection2.getZookeeper - val zwe = new ZKCheckedEphemeral(path, "", zk2, JaasUtils.isZkSecurityEnabled()) - - // Creates znode for path in the first session - zk1.create(path, Array[Byte](), Ids.OPEN_ACL_UNSAFE, CreateMode.EPHEMERAL) - - //Bootstraps the ZKWatchedEphemeral object - val gotException = - try { - zwe.create() - false - } catch { - case _: ZkNodeExistsException => true - } - Assert.assertTrue(gotException) - zkClient2.close() - } - - /** - * Tests if succeeds with znode from the same session - */ - @Test - def testSameSession(): Unit = { - val path = "/zwe-test" - val zk = zkUtils.zkConnection.getZookeeper - // Creates znode for path in the first session - zk.create(path, Array[Byte](), Ids.OPEN_ACL_UNSAFE, CreateMode.EPHEMERAL) - - val zwe = new ZKCheckedEphemeral(path, "", zk, JaasUtils.isZkSecurityEnabled()) - //Bootstraps the ZKWatchedEphemeral object - val gotException = - try { - zwe.create() - false - } catch { - case _: ZkNodeExistsException => 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 deleted file mode 100644 index 0cf836df68199..0000000000000 --- a/core/src/test/scala/unit/kafka/zk/ZKPathTest.scala +++ /dev/null @@ -1,130 +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.zk - -import kafka.consumer.ConsumerConfig -import kafka.utils.{TestUtils, ZkUtils} -import org.apache.kafka.common.config.ConfigException -import org.junit.Assert._ -import org.junit.Test - -class ZKPathTest extends ZooKeeperTestHarness { - - val path = "/some_dir" - val zkSessionTimeoutMs = 1000 - def zkConnectWithInvalidRoot: String = zkConnect + "/ghost" - - @Test - def testCreatePersistentPathThrowsException(): Unit = { - val config = new ConsumerConfig(TestUtils.createConsumerProperties(zkConnectWithInvalidRoot, - "test", "1")) - val zkUtils = ZkUtils(zkConnectWithInvalidRoot, zkSessionTimeoutMs, - config.zkConnectionTimeoutMs, false) - try { - zkUtils.zkPath.resetNamespaceCheckedState - zkUtils.createPersistentPath(path) - fail("Failed to throw ConfigException for missing ZooKeeper root node") - } catch { - case _: ConfigException => - } - zkUtils.close() - } - - @Test - def testCreatePersistentPath(): Unit = { - val config = new ConsumerConfig(TestUtils.createConsumerProperties(zkConnect, "test", "1")) - val zkUtils = ZkUtils(zkConnect, zkSessionTimeoutMs, config.zkConnectionTimeoutMs, false) - zkUtils.zkPath.resetNamespaceCheckedState - zkUtils.createPersistentPath(path) - assertTrue("Failed to create persistent path", zkUtils.pathExists(path)) - zkUtils.close() - } - - @Test - def testMakeSurePersistsPathExistsThrowsException(): Unit = { - val config = new ConsumerConfig(TestUtils.createConsumerProperties(zkConnectWithInvalidRoot, "test", "1")) - val zkUtils = ZkUtils(zkConnectWithInvalidRoot, zkSessionTimeoutMs, config.zkConnectionTimeoutMs, false) - try { - zkUtils.zkPath.resetNamespaceCheckedState - zkUtils.makeSurePersistentPathExists(path) - fail("Failed to throw ConfigException for missing ZooKeeper root node") - } catch { - case _: ConfigException => - } - zkUtils.close() - } - - @Test - def testMakeSurePersistsPathExists(): Unit = { - val config = new ConsumerConfig(TestUtils.createConsumerProperties(zkConnect, "test", "1")) - val zkUtils = ZkUtils(zkConnect, zkSessionTimeoutMs, config.zkConnectionTimeoutMs, false) - zkUtils.zkPath.resetNamespaceCheckedState - zkUtils.makeSurePersistentPathExists(path) - assertTrue("Failed to create persistent path", zkUtils.pathExists(path)) - zkUtils.close() - } - - @Test - def testCreateEphemeralPathThrowsException(): Unit = { - val config = new ConsumerConfig(TestUtils.createConsumerProperties(zkConnectWithInvalidRoot, "test", "1")) - val zkUtils = ZkUtils(zkConnectWithInvalidRoot, zkSessionTimeoutMs, config.zkConnectionTimeoutMs, false) - try { - zkUtils.zkPath.resetNamespaceCheckedState - zkUtils.createEphemeralPathExpectConflict(path, "somedata") - fail("Failed to throw ConfigException for missing ZooKeeper root node") - } catch { - case _: ConfigException => - } - zkUtils.close() - } - - @Test - def testCreateEphemeralPathExists(): Unit = { - val config = new ConsumerConfig(TestUtils.createConsumerProperties(zkConnect, "test", "1")) - val zkUtils = ZkUtils(zkConnect, zkSessionTimeoutMs, config.zkConnectionTimeoutMs, false) - zkUtils.zkPath.resetNamespaceCheckedState - zkUtils.createEphemeralPathExpectConflict(path, "somedata") - assertTrue("Failed to create ephemeral path", zkUtils.pathExists(path)) - zkUtils.close() - } - - @Test - def testCreatePersistentSequentialThrowsException(): Unit = { - val config = new ConsumerConfig(TestUtils.createConsumerProperties(zkConnectWithInvalidRoot, - "test", "1")) - val zkUtils = ZkUtils(zkConnectWithInvalidRoot, zkSessionTimeoutMs, config.zkConnectionTimeoutMs, false) - try { - zkUtils.zkPath.resetNamespaceCheckedState - zkUtils.createSequentialPersistentPath(path) - fail("Failed to throw ConfigException for missing ZooKeeper root node") - } catch { - case _: ConfigException => - } - zkUtils.close() - } - - @Test - def testCreatePersistentSequentialExists(): Unit = { - val config = new ConsumerConfig(TestUtils.createConsumerProperties(zkConnect, "test", "1")) - val zkUtils = ZkUtils(zkConnect, zkSessionTimeoutMs, config.zkConnectionTimeoutMs, false) - zkUtils.zkPath.resetNamespaceCheckedState - val actualPath = zkUtils.createSequentialPersistentPath(path) - assertTrue("Failed to create persistent path", zkUtils.pathExists(actualPath)) - zkUtils.close() - } -} diff --git a/core/src/test/scala/unit/kafka/zk/ZooKeeperTestHarness.scala b/core/src/test/scala/unit/kafka/zk/ZooKeeperTestHarness.scala index 034557e8a2ae8..8d34c489406df 100755 --- a/core/src/test/scala/unit/kafka/zk/ZooKeeperTestHarness.scala +++ b/core/src/test/scala/unit/kafka/zk/ZooKeeperTestHarness.scala @@ -32,6 +32,7 @@ import scala.collection.JavaConverters._ import org.apache.kafka.clients.producer.KafkaProducer import org.apache.kafka.clients.consumer.internals.AbstractCoordinator import kafka.controller.ControllerEventManager +import org.apache.kafka.clients.admin.AdminClientUnitTestEnv import org.apache.kafka.common.utils.Time import org.apache.zookeeper.{WatchedEvent, Watcher, ZooKeeper} @@ -91,6 +92,7 @@ object ZooKeeperTestHarness { // which reset static JAAS configuration. val unexpectedThreadNames = Set(ControllerEventManager.ControllerEventThreadName, KafkaProducer.NETWORK_THREAD_PREFIX, + AdminClientUnitTestEnv.kafkaAdminClientNetworkThreadPrefix(), AbstractCoordinator.HEARTBEAT_THREAD_PREFIX, ZkClientEventThreadPrefix) diff --git a/core/src/test/scala/unit/kafka/zookeeper/ZooKeeperClientTest.scala b/core/src/test/scala/unit/kafka/zookeeper/ZooKeeperClientTest.scala index c4143e2a8e6a6..0088c657de0a4 100644 --- a/core/src/test/scala/unit/kafka/zookeeper/ZooKeeperClientTest.scala +++ b/core/src/test/scala/unit/kafka/zookeeper/ZooKeeperClientTest.scala @@ -57,16 +57,26 @@ class ZooKeeperClientTest extends ZooKeeperTestHarness { System.clearProperty(JaasUtils.JAVA_LOGIN_CONFIG_PARAM) } - @Test(expected = classOf[IllegalArgumentException]) + @Test def testUnresolvableConnectString(): Unit = { - new ZooKeeperClient("some.invalid.hostname.foo.bar.local", -1, -1, Int.MaxValue, time, "testMetricGroup", - "testMetricType").close() + try { + new ZooKeeperClient("some.invalid.hostname.foo.bar.local", zkSessionTimeout, connectionTimeoutMs = 10, + Int.MaxValue, time, "testMetricGroup", "testMetricType") + } catch { + case e: ZooKeeperClientTimeoutException => + assertEquals("ZooKeeper client threads still running", Set.empty, runningZkSendThreads) + } } + private def runningZkSendThreads: collection.Set[String] = Thread.getAllStackTraces.keySet.asScala + .filter(_.isAlive) + .map(_.getName) + .filter(t => t.contains("SendThread()")) + @Test(expected = classOf[ZooKeeperClientTimeoutException]) def testConnectionTimeout(): Unit = { zookeeper.shutdown() - new ZooKeeperClient(zkConnect, zkSessionTimeout, connectionTimeoutMs = 100, Int.MaxValue, time, "testMetricGroup", + new ZooKeeperClient(zkConnect, zkSessionTimeout, connectionTimeoutMs = 10, Int.MaxValue, time, "testMetricGroup", "testMetricType").close() } diff --git a/docs/configuration.html b/docs/configuration.html index 8c86534fb154c..90c990bdaad86 100644 --- a/docs/configuration.html +++ b/docs/configuration.html @@ -90,6 +90,23 @@
    Updating Password Configs Dynamically
    using kafka-configs.sh even if the password config is not being altered. This constraint will be removed in a future release.

    +
    Updating Password Configs in ZooKeeper Before Starting Brokers
    + + From Kafka 2.0.0 onwards, kafka-configs.sh enables dynamic broker configs to be updated using ZooKeeper before + starting brokers for bootstrapping. This enables all password configs to be stored in encrypted form, avoiding the need for + clear passwords in server.properties. The broker config password.encoder.secret must also be specified + if any password configs are included in the alter command. Additional encryption parameters may also be specified. Password + encoder configs will not be persisted in ZooKeeper. For example, to store SSL key password for listener INTERNAL + on broker 0: + +
    +  > bin/kafka-configs.sh --zookeeper localhost:2181 --entity-type brokers --entity-name 0 --alter --add-config
    +    'listener.name.internal.ssl.key.password=key-password,password.encoder.secret=secret,password.encoder.iterations=8192'
    +  
    + + The configuration listener.name.internal.ssl.key.password will be persisted in ZooKeeper in encrypted + form using the provided encoder configs. The encoder secret and iterations are not persisted in ZooKeeper. +
    Updating SSL Keystore of an Existing Listener
    Brokers may be configured with SSL keystores with short validity periods to reduce the risk of compromised certificates. Keystores may be updated dynamically without restarting the broker. The config name must be prefixed with the listener prefix diff --git a/docs/design.html b/docs/design.html index 69d1941effd27..bdc7e637ea9b8 100644 --- a/docs/design.html +++ b/docs/design.html @@ -610,10 +610,12 @@

    Enforcemen having a fixed cluster wide bandwidth per client because that would require a mechanism to share client quota usage among all the brokers. This can be harder to get right than the quota implementation itself!

    - How does a broker react when it detects a quota violation? In our solution, the broker does not return an error rather it attempts to slow down a client exceeding its quota. - It computes the amount of delay needed to bring a guilty client under its quota and delays the response for that time. This approach keeps the quota violation transparent to clients - (outside of client-side metrics). This also keeps them from having to implement any special backoff and retry behavior which can get tricky. In fact, bad client behavior (retry without backoff) - can exacerbate the very problem quotas are trying to solve. + How does a broker react when it detects a quota violation? In our solution, the broker first computes the amount of delay needed to bring the violating client under its quota + and returns a response with the delay immediately. In case of a fetch request, the response will not contain any data. Then, the broker mutes the channel to the client, + not to process requests from the client anymore, until the delay is over. Upon receiving a response with a non-zero delay duration, the Kafka client will also refrain from + sending further requests to the broker during the delay. Therefore, requests from a throttled client are effectively blocked from both sides. + Even with older client implementations that do not respect the delay response from the broker, the back pressure applied by the broker via muting its socket channel + can still handle the throttling of badly behaving clients. Those clients who sent further requests to the throttled channel will receive responses only after the delay is over.

    Byte-rate and thread utilization are measured over multiple small windows (e.g. 30 windows of 1 second each) in order to detect and correct quota violations quickly. Typically, having large measurement windows diff --git a/docs/documentation.html b/docs/documentation.html index 80acf64588c1b..5dcf5c1b822c2 100644 --- a/docs/documentation.html +++ b/docs/documentation.html @@ -26,8 +26,8 @@

    Documentation

    -

    Kafka 1.1 Documentation

    - Prior releases:
    0.7.x, 0.8.0, 0.8.1.X, 0.8.2.X, 0.9.0.X, 0.10.0.X, 0.10.1.X, 0.10.2.X, 0.11.0.X, 1.0.X. +

    Kafka 2.0 Documentation

    + Prior releases: 0.7.x, 0.8.0, 0.8.1.X, 0.8.2.X, 0.9.0.X, 0.10.0.X, 0.10.1.X, 0.10.2.X, 0.11.0.X, 1.0.X, 1.1.X. diff --git a/docs/js/templateData.js b/docs/js/templateData.js index 28f0eec129cd6..2bf62ac0e7d8b 100644 --- a/docs/js/templateData.js +++ b/docs/js/templateData.js @@ -17,8 +17,8 @@ limitations under the License. // Define variables for doc templates var context={ - "version": "11", - "dotVersion": "1.1", - "fullDotVersion": "1.1.0", + "version": "20", + "dotVersion": "2.0", + "fullDotVersion": "2.0.0", "scalaVersion": "2.11" }; diff --git a/docs/ops.html b/docs/ops.html index 95b9a96017221..b9e3a4b32a126 100644 --- a/docs/ops.html +++ b/docs/ops.html @@ -1321,22 +1321,42 @@
    The average number of commit calls per second. kafka.streams:type=stream-task-metrics,client-id=([-.\w]+),task-id=([-.\w]+) + + commit-total + The total number of commit calls. + kafka.streams:type=stream-task-metrics,client-id=([-.\w]+),task-id=([-.\w]+) +
    Processor Node Metrics
    -All the following metrics have a recording level of ``debug``: + All the following metrics have a recording level of ``debug``: @@ -1443,33 +1478,60 @@
    + + + + + + + + + + + + + + + + + + + + + + + + +
    The average number of process operations per second. kafka.streams:type=stream-processor-node-metrics,client-id=([-.\w]+),task-id=([-.\w]+),processor-node-id=([-.\w]+)
    process-totalThe total number of process operations called. kafka.streams:type=stream-processor-node-metrics,client-id=([-.\w]+),task-id=([-.\w]+),processor-node-id=([-.\w]+)
    punctuate-rate The average number of punctuate operations per second. kafka.streams:type=stream-processor-node-metrics,client-id=([-.\w]+),task-id=([-.\w]+),processor-node-id=([-.\w]+)
    punctuate-totalThe total number of punctuate operations called. kafka.streams:type=stream-processor-node-metrics,client-id=([-.\w]+),task-id=([-.\w]+),processor-node-id=([-.\w]+)
    create-rate The average number of create operations per second. kafka.streams:type=stream-processor-node-metrics,client-id=([-.\w]+),task-id=([-.\w]+),processor-node-id=([-.\w]+)
    create-totalThe total number of create operations called. kafka.streams:type=stream-processor-node-metrics,client-id=([-.\w]+),task-id=([-.\w]+),processor-node-id=([-.\w]+)
    destroy-rate The average number of destroy operations per second. kafka.streams:type=stream-processor-node-metrics,client-id=([-.\w]+),task-id=([-.\w]+),processor-node-id=([-.\w]+)
    destroy-totalThe total number of destroy operations called. kafka.streams:type=stream-processor-node-metrics,client-id=([-.\w]+),task-id=([-.\w]+),processor-node-id=([-.\w]+)
    forward-rate The average rate of records being forwarded downstream, from source nodes only, per second. kafka.streams:type=stream-processor-node-metrics,client-id=([-.\w]+),task-id=([-.\w]+),processor-node-id=([-.\w]+)
    forward-totalThe total number of of records being forwarded downstream, from source nodes only. kafka.streams:type=stream-processor-node-metrics,client-id=([-.\w]+),task-id=([-.\w]+),processor-node-id=([-.\w]+)
    State Store Metrics
    -All the following metrics have a recording level of ``debug``: + All the following metrics have a recording level of ``debug``. Note that the ``store-scope`` value is specified in StoreSupplier#metricsScope() for user's customized + state stores; for built-in state stores, currently we have in-memory-state, in-memory-lru-state, rocksdb-state (for RocksDB backed key-value store), + rocksdb-window-state (for RocksDB backed window store) and rocksdb-session-state (for RocksDB backed session store). - +
    @@ -1479,143 +1541,188 @@
    Record Cache Metrics
    -All the following metrics have a recording level of ``debug``: + All the following metrics have a recording level of ``debug``:
    Metric/Attribute name
    diff --git a/docs/quickstart.html b/docs/quickstart.html index 063fec076b77d..e709cc736f4dc 100644 --- a/docs/quickstart.html +++ b/docs/quickstart.html @@ -113,12 +113,12 @@

    Step 6: Settin config/server-1.properties: broker.id=1 listeners=PLAINTEXT://:9093 - log.dir=/tmp/kafka-logs-1 + log.dirs=/tmp/kafka-logs-1 config/server-2.properties: broker.id=2 listeners=PLAINTEXT://:9094 - log.dir=/tmp/kafka-logs-2 + log.dirs=/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 other's data.

    diff --git a/docs/security.html b/docs/security.html index 57bba4775a33b..7f765090f6f01 100644 --- a/docs/security.html +++ b/docs/security.html @@ -547,27 +547,12 @@

    7.3 Authentication using SASL
  1. SASL/PLAIN should be used only with SSL as transport layer to ensure that clear passwords are not transmitted on the wire without encryption.
  2. The default implementation of SASL/PLAIN in Kafka specifies usernames and passwords in the JAAS configuration file as shown - here. To avoid storing passwords on disk, you can plug in your own implementation of - javax.security.auth.spi.LoginModule that provides usernames and passwords from an external source. The login module implementation should - provide username as the public credential and password as the private credential of the Subject. The default implementation - org.apache.kafka.common.security.plain.PlainLoginModule can be used as an example.
  3. -
  4. In production systems, external authentication servers may implement password authentication. Kafka brokers can be integrated with these servers by adding - your own implementation of javax.security.sasl.SaslServer. The default implementation included in Kafka in the package - org.apache.kafka.common.security.plain can be used as an example to get started. -
      -
    • New providers must be installed and registered in the JVM. Providers can be installed by adding provider classes to - the normal CLASSPATH or bundled as a jar file and added to JAVA_HOME/lib/ext.
    • -
    • Providers can be registered statically by adding a provider to the security properties file - JAVA_HOME/lib/security/java.security. -
          security.provider.n=providerClassName
      - where providerClassName is the fully qualified name of the new provider and n is the preference order with - lower numbers indicating higher preference.
    • -
    • Alternatively, you can register providers dynamically at runtime by invoking Security.addProvider at the beginning of the client - application or in a static initializer in the login module. For example: -
          Security.addProvider(new PlainSaslServerProvider());
    • -
    • For more details, see JCA Reference.
    • -
    -
  5. + here. From Kafka version 2.0 onwards, you can avoid storing clear passwords on disk + by configuring your own callback handlers that obtain username and password from an external source using the configuration options + sasl.server.callback.handler.class and sasl.client.callback.handler.class. +
  6. In production systems, external authentication servers may implement password authentication. From Kafka version 2.0 onwards, + you can plug in your own callback handlers that use external authentication servers for password verification by configuring + sasl.server.callback.handler.class.
  7. @@ -667,8 +652,8 @@

    7.3 Authentication using SASL
  8. SCRAM should be used only with TLS-encryption to prevent interception of SCRAM exchanges. This protects against dictionary or brute force attacks and against impersonation if Zookeeper is compromised.
  9. -
  10. The default SASL/SCRAM implementation may be overridden using custom login modules in installations - where Zookeeper is not secure. See here for details.
  11. +
  12. From Kafka version 2.0 onwards, the default SASL/SCRAM credential store may be overridden using custom callback handlers + by configuring sasl.server.callback.handler.class in installations where Zookeeper is not secure.
  13. For more details on security considerations, refer to RFC 5802. @@ -730,6 +715,8 @@

    7.3 Authentication using SASL security.protocol=SASL_SSL (or SASL_PLAINTEXT if non-production) sasl.mechanism=OAUTHBEARER

  14. +
  15. The default implementation of SASL/OAUTHBEARER depends on the jackson-databind library. + Since it's an optional dependency, users have to configure it as a dependency via their build tool.
  16. Unsecured Token Creation Options for SASL/OAUTHBEARER
    @@ -1022,7 +1009,7 @@

    7.3 Authentication using SASL7.4 Authorization and ACLs

    Kafka ships with a pluggable Authorizer and an out-of-box authorizer implementation that uses zookeeper to store all the acls. The Authorizer is configured by setting authorizer.class.name in server.properties. To enable the out of the box implementation use:
    authorizer.class.name=kafka.security.auth.SimpleAclAuthorizer
    - Kafka acls are defined in the general format of "Principal P is [Allowed/Denied] Operation O From Host H On Resource R". You can read more about the acl structure on KIP-11. In order to add, remove or list acls you can use the Kafka authorizer CLI. By default, if a Resource R has no associated acls, no one other than super users is allowed to access R. If you want to change that behavior, you can include the following in server.properties. + Kafka acls are defined in the general format of "Principal P is [Allowed/Denied] Operation O From Host H on any Resource R matching ResourcePattern RP". You can read more about the acl structure in KIP-11 and resource patterns in KIP-290. In order to add, remove or list acls you can use the Kafka authorizer CLI. By default, if no ResourcePatterns match a specific Resource R, then R has no associated acls, and therefore no one other than super users is allowed to access R. If you want to change that behavior, you can include the following in server.properties.
    allow.everyone.if.no.acl.found=true
    One can also add super users in server.properties like the following (note that the delimiter is semicolon since SSL user names may contain comma).
    super.users=User:Bob;User:Alice
    @@ -1083,31 +1070,30 @@

    Command Line Interface

  17. - + - + - + - + - + - + - - + @@ -1173,27 +1159,31 @@

    Examples By default, all principals that don't have an explicit acl that allows access for an operation to a resource are denied. In rare cases where an allow acl is defined that allows access to all but some principal we will have to use the --deny-principal and --deny-host option. For example, if we want to allow all users to Read from Test-topic but only deny User:BadBob from IP 198.51.100.3 we can do so using following commands:
    bin/kafka-acls.sh --authorizer-properties zookeeper.connect=localhost:2181 --add --allow-principal User:* --allow-host * --deny-principal User:BadBob --deny-host 198.51.100.3 --operation Read --topic Test-topic
    Note that ``--allow-host`` and ``deny-host`` only support IP addresses (hostnames are not supported). - Above examples add acls to a topic by specifying --topic [topic-name] as the resource option. Similarly user can add acls to cluster by specifying --cluster and to a consumer group by specifying --group [group-name]. + Above examples add acls to a topic by specifying --topic [topic-name] as the resource pattern option. Similarly user can add acls to cluster by specifying --cluster and to a consumer group by specifying --group [group-name]. You can add acls on any resource of a certain type, e.g. suppose you wanted to add an acl "Principal User:Peter is allowed to produce to any Topic from IP 198.51.200.0" You can do that by using the wildcard resource '*', e.g. by executing the CLI with following options:
    bin/kafka-acls.sh --authorizer-properties zookeeper.connect=localhost:2181 --add --allow-principal User:Peter --allow-host 198.51.200.1 --producer --topic *
    - You can add acls on resources matching a certain prefix, e.g. suppose you want to add an acl "Principal User:Jane is allowed to produce to any Topic whose name is prefixed with 'Test-' from any host". + You can add acls on prefixed resource patterns, e.g. suppose you want to add an acl "Principal User:Jane is allowed to produce to any Topic whose name starts with 'Test-' from any host". You can do that by executing the CLI with following options: -
    bin/kafka-acls.sh --authorizer-properties zookeeper.connect=localhost:2181 --add --allow-principal User:Jane --producer --topic Test- --resource-name-type Prefixed
    - Note, --resource-name-type defaults to 'literal', which only affects resources with the exact same name. The exception to this is the wildcard resource name '*', which should also be added using 'literal'. +
    bin/kafka-acls.sh --authorizer-properties zookeeper.connect=localhost:2181 --add --allow-principal User:Jane --producer --topic Test- --resource-pattern-type prefixed
    + Note, --resource-pattern-type defaults to 'literal', which only affects resources with the exact same name or, in the case of the wildcard resource name '*', a resource with any name.
  18. Removing Acls
    Removing acls is pretty much the same. The only difference is instead of --add option users will have to specify --remove option. To remove the acls added by the first example above we can execute the CLI with following options: -
     bin/kafka-acls.sh --authorizer-properties zookeeper.connect=localhost:2181 --remove --allow-principal User:Bob --allow-principal User:Alice --allow-host 198.51.100.0 --allow-host 198.51.100.1 --operation Read --operation Write --topic Test-topic 
  19. - If you wan to remove the prefixed acl added above we can execute the CLI with following options: -
     bin/kafka-acls.sh --authorizer-properties zookeeper.connect=localhost:2181 --remove --allow-principal User:Jane --producer --topic Test- --resource-name-type Prefixed
    +
     bin/kafka-acls.sh --authorizer-properties zookeeper.connect=localhost:2181 --remove --allow-principal User:Bob --allow-principal User:Alice --allow-host 198.51.100.0 --allow-host 198.51.100.1 --operation Read --operation Write --topic Test-topic 
    + If you wan to remove the acl added to the prefixed resource pattern above we can execute the CLI with following options: +
     bin/kafka-acls.sh --authorizer-properties zookeeper.connect=localhost:2181 --remove --allow-principal User:Jane --producer --topic Test- --resource-pattern-type Prefixed
  20. List Acls
    - We can list acls for any resource by specifying the --list option with the resource. To list all acls for Test-topic we can execute the CLI with following options: -
    bin/kafka-acls.sh --authorizer-properties zookeeper.connect=localhost:2181 --list --topic Test-topic
  21. - However, this will only return the acls that have been added to this exact resource. Other acls can exist that affect access to the topic, - e.g. any acls on the topic wildcard '*', or any acls on resources matching a certain prefix. To list all acls affecting a topic we can use the '--resource-name-type any' option, e.g. -
    bin/kafka-acls.sh --authorizer-properties zookeeper.connect=localhost:2181 --list --topic Test-topic --resource-name-type any
    + We can list acls for any resource by specifying the --list option with the resource. To list all acls on the literal resource pattern Test-topic, we can execute the CLI with following options: +
    bin/kafka-acls.sh --authorizer-properties zookeeper.connect=localhost:2181 --list --topic Test-topic
    + However, this will only return the acls that have been added to this exact resource pattern. Other acls can exist that affect access to the topic, + e.g. any acls on the topic wildcard '*', or any acls on prefixed resource patterns. Acls on the wildcard resource pattern can be queried explicitly: +
    bin/kafka-acls.sh --authorizer-properties zookeeper.connect=localhost:2181 --list --topic *
    + However, it is not necessarily possible to explicitly query for acls on prefixed resource patterns that match Test-topic as the name of such patterns may not be known. + We can list all acls affecting Test-topic by using '--resource-pattern-type match', e.g. +
    bin/kafka-acls.sh --authorizer-properties zookeeper.connect=localhost:2181 --list --topic Test-topic --resource-pattern-type match
    + This will list acls on all matching literal, wildcard and prefixed resource patterns.
  22. Adding or removing a principal as producer or consumer
    The most common use case for acl management are adding/removing a principal as producer or consumer so we added convenience options to handle these cases. In order to add User:Bob as a producer of Test-topic we can execute the following command: diff --git a/docs/streams/core-concepts.html b/docs/streams/core-concepts.html index 3f9eab57ecdbd..4e5a7ae6511e1 100644 --- a/docs/streams/core-concepts.html +++ b/docs/streams/core-concepts.html @@ -63,7 +63,7 @@

    Stream Processing Topology There are two special processors in the topology: @@ -131,13 +131,60 @@

    Time

    Note, that the describe default behavior can be changed in the Processor API by assigning timestamps to output records explicitly when calling #forward().

    +

    Aggregations

    +

    + An aggregation operation takes one input stream or table, and yields a new table by combining multiple input records into a single output record. Examples of aggregations are computing counts or sum. +

    + +

    + In the Kafka Streams DSL, an input stream of an aggregation can be a KStream or a KTable, but the output stream will always be a KTable. This allows Kafka Streams to update an aggregate value upon the late arrival of further records after the value was produced and emitted. When such late arrival happens, the aggregating KStream or KTable emits a new aggregate value. Because the output is a KTable, the new value is considered to overwrite the old value with the same key in subsequent processing steps. +

    + +

    Windowing

    +

    + Windowing lets you control how to group records that have the same key for stateful operations such as aggregations or joins into so-called windows. Windows are tracked per record key. +

    +

    + Windowing operations are available in the Kafka Streams DSL. When working with windows, you can specify a retention period for the window. This retention period controls how long Kafka Streams will wait for out-of-order or late-arriving data records for a given window. If a record arrives after the retention period of a window has passed, the record is discarded and will not be processed in that window. +

    +

    + Late-arriving records are always possible in the real world and should be properly accounted for in your applications. It depends on the effective time semantics how late records are handled. In the case of processing-time, the semantics are "when the record is being processed", which means that the notion of late records is not applicable as, by definition, no record can be late. Hence, late-arriving records can only be considered as such (i.e. as arriving "late") for event-time or ingestion-time semantics. In both cases, Kafka Streams is able to properly handle late-arriving records. +

    + +

    Duality of Streams and Tables

    +

    + When implementing stream processing use cases in practice, you typically need both streams and also databases. + An example use case that is very common in practice is an e-commerce application that enriches an incoming stream of customer + transactions with the latest customer information from a database table. In other words, streams are everywhere, but databases are everywhere, too. +

    + +

    + Any stream processing technology must therefore provide first-class support for streams and tables. + Kafka's Streams API provides such functionality through its core abstractions for + streams + and tables, + which we will talk about in a minute. Now, an interesting observation is that there is actually a close relationship between streams and tables, + the so-called stream-table duality. And Kafka exploits this duality in many ways: for example, to make your applications + elastic, + to support fault-tolerant stateful processing, + or to run interactive queries + against your application's latest processing results. And, beyond its internal usage, the Kafka Streams API + also allows developers to exploit this duality in their own applications. +

    + +

    + Before we discuss concepts such as aggregations + in Kafka Streams, we must first introduce tables in more detail, and talk about the aforementioned stream-table duality. + Essentially, this duality means that a stream can be viewed as a table, and a table can be viewed as a stream. +

    +

    States

    Some stream processing applications don't require state, which means the processing of a message is independent from the processing of all other messages. However, being able to maintain state opens up many possibilities for sophisticated stream processing applications: you - can join input streams, or group and aggregate data records. Many such stateful operators are provided by the Kafka Streams DSL. + can join input streams, or group and aggregate data records. Many such stateful operators are provided by the Kafka Streams DSL.

    Kafka Streams provides so-called state stores, which can be used by stream processing applications to store and query data. diff --git a/docs/streams/developer-guide/app-reset-tool.html b/docs/streams/developer-guide/app-reset-tool.html index 84b6930f10a44..2f82cf13ae4a0 100644 --- a/docs/streams/developer-guide/app-reset-tool.html +++ b/docs/streams/developer-guide/app-reset-tool.html @@ -141,7 +141,7 @@

    Step 1: Run the application reset tool
  23. The API method KafkaStreams#cleanUp() in your application code.
  24. -
  25. Manually delete the corresponding local state directory (default location: /var/lib/kafka-streams/<application.id>). For more information, see Streams javadocs.
  26. +
  27. Manually delete the corresponding local state directory (default location: /tmp/kafka-streams/<application.id>). For more information, see Streams javadocs.
  28. diff --git a/docs/streams/developer-guide/config-streams.html b/docs/streams/developer-guide/config-streams.html index 6bba10d821088..47b908292c570 100644 --- a/docs/streams/developer-guide/config-streams.html +++ b/docs/streams/developer-guide/config-streams.html @@ -272,12 +272,12 @@

    bootstrap.servers

  29. - + - + @@ -300,8 +300,10 @@

    bootstrap.servers

    default.deserialization.exception.handler

    The default deserialization exception handler allows you to manage record exceptions that fail to deserialize. This - can be caused by corrupt data, incorrect serialization logic, or unhandled record types. These exception handlers - are available:

    + can be caused by corrupt data, incorrect serialization logic, or unhandled record types. The implemented exception + handler needs to return a FAIL or CONTINUE depending on the record and the exception thrown. Returning + FAIL will signal that Streams should shut down and CONTINUE will signal that Streams should ignore the issue + and continue processing. The following library built-in exception handlers are available:

    + +

    You can also provide your own customized exception handler besides the library provided ones to meet your needs. For example, you can choose to forward corrupt + records into a quarantine topic (think: a "dead letter queue") for further processing. To do this, use the Producer API to write a corrupted record directly to + the quarantine topic. To be more concrete, you can create a separate KafkaProducer object outside the Streams client, and pass in this object + as well as the dead letter queue topic name into the Properties map, which then can be retrieved from the configure function call. + The drawback of this approach is that "manual" writes are side effects that are invisible to the Kafka Streams runtime library, + so they do not benefit from the end-to-end processing guarantees of the Streams API:

    + +
    +              public class SendToDeadLetterQueueExceptionHandler implements DeserializationExceptionHandler {
    +                  KafkaProducer<byte[], byte[]> dlqProducer;
    +                  String dlqTopic;
    +
    +                  @Override
    +                  public DeserializationHandlerResponse handle(final ProcessorContext context,
    +                                                               final ConsumerRecord<byte[], byte[]> record,
    +                                                               final Exception exception) {
    +
    +                      log.warn("Exception caught during Deserialization, sending to the dead queue topic; " +
    +                          "taskId: {}, topic: {}, partition: {}, offset: {}",
    +                          context.taskId(), record.topic(), record.partition(), record.offset(),
    +                          exception);
    +
    +                      dlqProducer.send(new ProducerRecord<>(dlqTopic, record.timestamp(), record.key(), record.value(), record.headers())).get();
    +
    +                      return DeserializationHandlerResponse.CONTINUE;
    +                  }
    +
    +                  @Override
    +                  public void configure(final Map<String, ?> configs) {
    +                      dlqProducer = .. // get a producer from the configs map
    +                      dlqTopic = .. // get the topic name from the configs map
    +                  }
    +              }
    +              
    +
    Properties streamsSettings = new Properties();
     streamsSettings.put(StreamsConfig.REPLICATION_FACTOR_CONFIG, 3);
    +streamsSettings.put(StreamsConfig.topicPrefix(TopicConfig.MIN_IN_SYNC_REPLICAS_CONFIG), 2);
     streamsSettings.put(StreamsConfig.producerPrefix(ProducerConfig.ACKS_CONFIG), "all");
     
    diff --git a/docs/streams/developer-guide/datatypes.html b/docs/streams/developer-guide/datatypes.html index 11208154fae9b..314d6ee124082 100644 --- a/docs/streams/developer-guide/datatypes.html +++ b/docs/streams/developer-guide/datatypes.html @@ -49,7 +49,7 @@
  30. Primitive and basic types
  31. Avro
  32. JSON
  33. -
  34. Further serdes
  35. +
  36. Implementing custom serdes
  37. Kafka Streams DSL for Scala Implicit SerDes
  38. @@ -93,6 +93,11 @@

    Overriding default SerDesuserCountByRegion.to("RegionCountsTopic", Produced.valueSerde(Serdes.Long())); +

    If some of your incoming records are corrupted or ill-formatted, they will cause the deserializer class to report an error. + Since 1.0.x we have introduced an DeserializationExceptionHandler interface which allows + you to customize how to handle such records. The customized implementation of the interface can be specified via the StreamsConfig. + For more details, please feel free to read the Configuring a Streams Application section. +

    Available SerDes

    diff --git a/docs/streams/developer-guide/dsl-api.html b/docs/streams/developer-guide/dsl-api.html index cd3a9656d127e..49af92aed2f5e 100644 --- a/docs/streams/developer-guide/dsl-api.html +++ b/docs/streams/developer-guide/dsl-api.html @@ -79,9 +79,9 @@

    Overview

    In comparison to the Processor API, only the DSL supports:

      -
    • Built-in abstractions for streams and tables in the form of - KStream, KTable, and - GlobalKTable. Having first-class support for streams and tables is crucial +
    • Built-in abstractions for streams and tables in the form of + KStream, KTable, and + GlobalKTable. Having first-class support for streams and tables is crucial because, in practice, most use cases require not just either streams or databases/tables, but a combination of both. For example, if your use case is to create a customer 360-degree view that is updated in real-time, what your application will be doing is transforming many input streams of customer-related events into an output table @@ -93,7 +93,7 @@

      Overviewjoins (e.g. leftJoin), and windowing (e.g. session windows).

    -

    With the DSL, you can define processor topologies (i.e., the logical +

    With the DSL, you can define processor topologies (i.e., the logical processing plan) in your application. The steps to accomplish this are:

    1. Specify one or more input streams that are read from Kafka topics.
    2. @@ -104,6 +104,113 @@

      OverviewStreams API docs.

    + +
    +

    KStream

    + +

    + Only the Kafka Streams DSL has the notion of a KStream. +

    + +

    + A KStream is an abstraction of a record stream, where each data record represents a self-contained datum in the unbounded data set. Using the table analogy, data records in a record stream are always interpreted as an "INSERT" -- think: adding more entries to an append-only ledger -- because no record replaces an existing row with the same key. Examples are a credit card transaction, a page view event, or a server log entry. +

    + +

    + To illustrate, let's imagine the following two data records are being sent to the stream: +

    + +
    +

    ("alice", 1) --> ("alice", 3)

    +
    + +

    + If your stream processing application were to sum the values per user, it would return 4 for alice. Why? Because the second data record would not be considered an update of the previous record. Compare this behavior of KStream to KTable below, + which would return 3 for alice. +

    + +

    KTable

    + +

    + Only the Kafka Streams DSL has the notion of a KTable. +

    + +

    + A KTable is an abstraction of a changelog stream, where each data record represents an update. More precisely, the value in a data record is interpreted as an "UPDATE" of the last value for the same record key, if any (if a corresponding key doesn't exist yet, the update will be considered an INSERT). Using the table analogy, a data record in a changelog stream is interpreted as an UPSERT aka INSERT/UPDATE because any existing row with the same key is overwritten. Also, null values are interpreted in a special way: a record with a null value represents a "DELETE" or tombstone for the record's key. +

    +

    + To illustrate, let's imagine the following two data records are being sent to the stream: +

    + +
    +

    + ("alice", 1) --> ("alice", 3) +

    +
    + +

    + If your stream processing application were to sum the values per user, it would return 3 for alice. Why? Because the second data record would be considered an update of the previous record. +

    + +

    + Effects of Kafka's log compaction: Another way of thinking about KStream and KTable is as follows: If you were to store a KTable into a Kafka topic, you'd probably want to enable Kafka's log compaction feature, e.g. to save storage space. +

    + +

    + However, it would not be safe to enable log compaction in the case of a KStream because, as soon as log compaction would begin purging older data records of the same key, it would break the semantics of the data. To pick up the illustration example again, you'd suddenly get a 3 for alice instead of a 4 because log compaction would have removed the ("alice", 1) data record. Hence log compaction is perfectly safe for a KTable (changelog stream) but it is a mistake for a KStream (record stream). +

    + +

    + We have already seen an example of a changelog stream in the section streams and tables. Another example are change data capture (CDC) records in the changelog of a relational database, representing which row in a database table was inserted, updated, or deleted. +

    + +

    + KTable also provides an ability to look up current values of data records by keys. This table-lookup functionality is available through join operations (see also Joining in the Developer Guide) as well as through Interactive Queries. +

    + +

    GlobalKTable

    + +

    Only the Kafka Streams DSL has the notion of a GlobalKTable.

    + +

    + Like a KTable, a GlobalKTable is an abstraction of a changelog stream, where each data record represents an update. +

    + +

    + A GlobalKTable differs from a KTable in the data that they are being populated with, i.e. which data from the underlying Kafka topic is being read into the respective table. Slightly simplified, imagine you have an input topic with 5 partitions. In your application, you want to read this topic into a table. Also, you want to run your application across 5 application instances for maximum parallelism. +

    + +
      +
    • + If you read the input topic into a KTable, then the "local" KTable instance of each application instance will be populated with data from only 1 partition of the topic's 5 partitions. +
    • + +
    • + If you read the input topic into a GlobalKTable, then the local GlobalKTable instance of each application instance will be populated with data from all partitions of the topic. +
    • +
    + +

    + GlobalKTable provides the ability to look up current values of data records by keys. This table-lookup functionality is available through join operations. +

    +

    Benefits of global tables:

    + +
      +
    • + More convenient and/or efficient joins: Notably, global tables allow you to perform star joins, they support "foreign-key" lookups (i.e., you can lookup data in the table not just by record key, but also by data in the record values), and they are more efficient when chaining multiple joins. Also, when joining against a global table, the input data does not need to be co-partitioned. +
    • +
    • + Can be used to "broadcast" information to all the running instances of your application. +
    • +
    + +

    Downsides of global tables:

    +
      +
    • Increased local storage consumption compared to the (partitioned) KTable because the entire topic is tracked.
    • +
    • Increased network and Kafka broker load compared to the (partitioned) KTable because the entire topic is read.
    • +
    + +

    Creating source streams from Kafka

    You can easily read data from Kafka topics into your application. The following operations are supported.

    @@ -123,8 +230,8 @@

    OverviewCreates a KStream from the specified Kafka input topics and interprets the data - as a record stream. +

    @@ -2897,7 +3004,7 @@

    Overviewtransform will result in re-partitioning of the records. If possible use transformValues instead, which will not cause data re-partitioning.

    transform is essentially equivalent to adding the Transformer via Topology#addProcessor() to your - processor topology.

    + processor topology.

    An example is available in the javadocs.

    @@ -2916,7 +3023,7 @@

    OverviewValueTransformer may return null as the new value for a record.

    transformValues is preferable to transform because it will not cause data re-partitioning.

    transformValues is essentially equivalent to adding the ValueTransformer via Topology#addProcessor() to your - processor topology.

    + processor topology.

    An example is available in the javadocs.

    @@ -3183,6 +3290,33 @@

    Overview libraryDependencies += "org.apache.kafka" %% "kafka-streams-scala" % "{{fullDotVersion}}" +

    + Notes: +

      +
    • + The bugfix version 2.0.1 fixed several important flaws in kafka-streams-scala. + We strongly recommend that you use the latest bugfix release in general and avoid kafka-streams-scala 2.0.0 specifically. +
    • +
    • +

      + There is an upstream dependency that causes trouble in SBT builds. This issue is only present in 2.0.0, 2.0.1, and 2.1.0 and is fixed in subsequent major, minor, and bugfix releases. +

      +

      + If you must use an affected version, add an explicit dependency on the problematic library as a workaround: +

      +
      2.0.0
      +
      libraryDependencies += "javax.ws.rs" % "javax.ws.rs-api" % "2.1" artifacts(Artifact("javax.ws.rs-api", "jar", "jar"))
      +
      2.0.1
      +
      libraryDependencies += "javax.ws.rs" % "javax.ws.rs-api" % "2.1" artifacts(Artifact("javax.ws.rs-api", "jar", "jar"))
      +
      2.1.0
      +
      libraryDependencies += "javax.ws.rs" % "javax.ws.rs-api" % "2.1.1" artifacts(Artifact("javax.ws.rs-api", "jar", "jar"))
      +
      (any later release)
      +
      No workaround needed
      +
      +

      +
    • +
    +

    Sample Usage

    The library works by wrapping the original Java abstractions of Kafka Streams within a Scala wrapper object and then using implicit conversions between them. All the Scala abstractions are named identically as the corresponding Java abstraction, but they reside in a different package of the library e.g. the Scala class org.apache.kafka.streams.scala.StreamsBuilder is a wrapper around org.apache.kafka.streams.StreamsBuilder, org.apache.kafka.streams.scala.kstream.KStream is a wrapper around org.apache.kafka.streams.kstream.KStream, and so on.

    diff --git a/docs/streams/developer-guide/interactive-queries.html b/docs/streams/developer-guide/interactive-queries.html index 051f87c87d531..6c553a7d9fd5d 100644 --- a/docs/streams/developer-guide/interactive-queries.html +++ b/docs/streams/developer-guide/interactive-queries.html @@ -59,7 +59,7 @@

    There are local and remote components to interactively querying the state of your application.

    Local state
    -
    An application instance can query the locally managed portion of the state and directly query its own local state stores. You can use the corresponding local data in other parts of your application code, as long as it doesn’t required calling the Kafka Streams API. Querying state stores is always read-only to guarantee that the underlying state stores will never be mutated out-of-band (e.g., you cannot add new entries). State stores should only be mutated by the corresponding processor topology and the input data it operates on. For more information, see Querying local state stores for an app instance.
    +
    An application instance can query the locally managed portion of the state and directly query its own local state stores. You can use the corresponding local data in other parts of your application code, as long as it doesn’t require calling the Kafka Streams API. Querying state stores is always read-only to guarantee that the underlying state stores will never be mutated out-of-band (e.g., you cannot add new entries). State stores should only be mutated by the corresponding processor topology and the input data it operates on. For more information, see Querying local state stores for an app instance.
    Remote state

    To query the full state of your application, you must connect the various fragments of the state, including:

    Accessing Processor Context

    -

    As we have mentioned in Defining a Stream Processor<\href>, a ProcessorContext control the processing workflow, such as scheduling a punctuation function, and committing the current processed state.

    +

    As we have mentioned in the Defining a Stream Processor section, a ProcessorContext control the processing workflow, such as scheduling a punctuation function, and committing the current processed state.

    This object can also be used to access the metadata related with the application like applicationId, taskId, and stateDir, and also record related metadata as topic, @@ -363,7 +363,7 @@

    Accessing Processor ContextHere is an example implementation of how to add a new header to the record:

    public void process(String key, String value) {
                 
    -            // add a heaeder to the elements
    +            // add a header to the elements
                 context().headers().add.("key", "key"
                 }
                 
    diff --git a/docs/streams/developer-guide/security.html b/docs/streams/developer-guide/security.html index 9c494566b6f21..e196e14c1f8d2 100644 --- a/docs/streams/developer-guide/security.html +++ b/docs/streams/developer-guide/security.html @@ -64,17 +64,20 @@

    For more information about the security features in Apache Kafka, see Kafka Security.

    Required ACL setting for secure Kafka clusters

    -

    When applications are run against a secured Kafka cluster, the principal running the application must have the ACL +

    Kafka clusters can use ACLs to control access to resources (like the ability to create topics), and for such clusters each client, + including Kafka Streams, is required to authenticate as a particular user in order to be authorized with appropriate access. + In particular, when Streams applications are run against a secured Kafka cluster, the principal running the application must have + the ACL --cluster --operation Create set so that the application has the permissions to create internal topics.

    -

    To avoid providing this permission to your application, you can create the required internal topics manually. - If the internal topics exist, Kafka Streams will not try to recreate them. - Note, that the internal repartition and changelog topics must be created with the correct number of partitions—otherwise, Kafka Streams will fail on startup. - The topics must be created with the same number of partitions as your input topic, or if there are multiple topics, the maximum number of partitions across all input topics. - Additionally, changelog topics must be created with log compaction enabled—otherwise, your application might lose data. - You can find out more about the names of the required internal topics via Topology#describe(). - All internal topics follow the naming pattern <application.id>-<operatorName>-<suffix> where the suffix is either repartition or changelog. - Note, that there is no guarantee about this naming pattern in future releases—it's not part of the public API.

    + +

    Since all internal topics as well as the embedded consumer group name are prefixed with the application id, + it is recommended to use ACLs on prefixed resource pattern + to configure control lists to allow client to manage all topics and consumer groups started with this prefix + (see KIP-277 + and KIP-290 for details). + For example, given the following setup of your Streams application: +

    diff --git a/docs/streams/developer-guide/write-streams.html b/docs/streams/developer-guide/write-streams.html index 145eb307a4f86..601f12a910f09 100644 --- a/docs/streams/developer-guide/write-streams.html +++ b/docs/streams/developer-guide/write-streams.html @@ -53,7 +53,7 @@

    Libraries and Maven artifacts

    This section lists the Kafka Streams related libraries that are available for writing your Kafka Streams applications.

    You can define dependencies on the following libraries for your Kafka Streams applications.

    -

    --clusterSpecifies cluster as resource.Indicates to the script that the user is trying to interact with acls on the singular cluster resource. ResourceResourcePattern
    --topic [topic-name]Specifies the topic as resource.Indicates to the script that the user is trying to interact with acls on topic resource pattern(s). ResourceResourcePattern
    --group [group-name]Specifies the consumer-group as resource.Indicates to the script that the user is trying to interact with acls on consumer-group resource pattern(s) ResourceResourcePattern
    --resource-name-type [name-type]Specifies the resource name type to use.
    - Valid values are:
    -
      -
    • Literal Match resource names exactly or, in the case of the Wildcard name '*', match all resources.
    • -
    • Prefixed Match any resource whose name starts with the prefix.
    • -
    • All (list|remove only) Matching any name type, including the Wildcard name.
    • -
    +
    --resource-pattern-type [pattern-type]Indicates to the script the type of resource pattern, (for --add), or resource pattern filter, (for --list and --remove), the user wishes to use.
    + When adding acls, this should be a specific pattern type, e.g. 'literal' or 'prefixed'.
    + When listing or removing acls, a specific pattern type filter can be used to list or remove acls from a specific type of resource pattern, + or the filter values of 'any' or 'match' can be used, where 'any' will match any pattern type, but will match the resource name exactly, + and 'match' will perform pattern matching to list or remove all acls that affect the supplied resource(s).
    + WARNING: 'match', when used in combination with the '--remove' switch, should be used with care.
    literal Configurationstate.cleanup.delay.ms Low The amount of time in milliseconds to wait before deleting state when a partition has migrated.6000000 milliseconds600000 milliseconds
    state.dir High Directory location for state stores./var/lib/kafka-streams/tmp/kafka-streams
    timestamp.extractor Medium

    Creates a KStream from the specified Kafka input topics and interprets the data + as a record stream. A KStream represents a partitioned record stream. (details)

    In the case of a KStream, the local KStream instance of every application instance will @@ -157,7 +264,7 @@

    OverviewReads the specified Kafka input topic into a KTable. The topic is +

    Reads the specified Kafka input topic into a KTable. The topic is interpreted as a changelog stream, where records with the same key are interpreted as UPSERT aka INSERT/UPDATE (when the record value is not null) or as DELETE (when the value is null) for that key. (details)

    @@ -182,7 +289,7 @@

    OverviewReads the specified Kafka input topic into a GlobalKTable. The topic is +

    Reads the specified Kafka input topic into a GlobalKTable. The topic is interpreted as a changelog stream, where records with the same key are interpreted as UPSERT aka INSERT/UPDATE (when the record value is not null) or as DELETE (when the value is null) for that key. (details)

    @@ -225,7 +332,7 @@

    Overviewfilter and map on a KStream will generate another KStream - branch on KStream can generate multiple KStreams

    -

    Some others may generate a KTable object, for example an aggregation of a KStream also yields a KTable. This allows Kafka Streams to continuously update the computed value upon arrivals of late records after it +

    Some others may generate a KTable object, for example an aggregation of a KStream also yields a KTable. This allows Kafka Streams to continuously update the computed value upon arrivals of late records after it has already been produced to the downstream transformation operators.

    All KTable transformation operations can only generate another KTable. However, the Kafka Streams DSL does provide a special function that converts a KTable representation into a KStream. All of these transformation methods can be chained together to compose @@ -898,9 +1005,9 @@

    Overview// Aggregating with time-based windowing (here: with 5-minute tumbling windows) KTable<Windowed<String>, Long> timeWindowedAggregatedStream = groupedStream.windowedBy(TimeUnit.MINUTES.toMillis(5)) .aggregate( - () -> 0L, /* initializer */ - (aggKey, newValue, aggValue) -> aggValue + newValue, /* adder */ - Materialized.<String, Long, WindowStore<Bytes, byte[]>>as("time-windowed-aggregated-stream-store") /* state store name */ + () -> 0L, /* initializer */ + (aggKey, newValue, aggValue) -> aggValue + newValue, /* adder */ + Materialized.<String, Long, WindowStore<Bytes, byte[]>>as("time-windowed-aggregated-stream-store") /* state store name */ .withValueSerde(Serdes.Long())); /* serde for aggregate value */ // Aggregating with session-based windowing (here: with an inactivity gap of 5 minutes) @@ -908,8 +1015,8 @@

    Overviewaggregate( () -> 0L, /* initializer */ (aggKey, newValue, aggValue) -> aggValue + newValue, /* adder */ - (aggKey, leftAggValue, rightAggValue) -> leftAggValue + rightAggValue, /* session merger */ - Materialized.<String, Long, SessionStore<Bytes, byte[]>>as("sessionized-aggregated-stream-store") /* state store name */ + (aggKey, leftAggValue, rightAggValue) -> leftAggValue + rightAggValue, /* session merger */ + Materialized.<String, Long, SessionStore<Bytes, byte[]>>as("sessionized-aggregated-stream-store") /* state store name */ .withValueSerde(Serdes.Long())); /* serde for aggregate value */ // Java 7 examples @@ -1461,7 +1568,7 @@

    OverviewKafka’s Connect API, and then implementing + change data capture in combination with Kafka’s Connect API, and then implementing applications that leverage the Streams API to perform very fast and efficient local joins of such tables and streams, rather than requiring the application to make a query to a remote database over the network for each record. In this example, the KTable concept in Kafka Streams would enable you to track the latest state @@ -1522,7 +1629,7 @@

    Overview

    Tip

    -

    If possible, consider using global tables (GlobalKTable) for joining because they do not require data co-partitioning.

    +

    If possible, consider using global tables (GlobalKTable) for joining because they do not require data co-partitioning.

    The requirements for data co-partitioning are:

      @@ -1530,7 +1637,7 @@

      OverviewJava Producer API must use the + This means that, for example, applications that use Kafka’s Java Producer API must use the same partitioner (cf. the producer setting "partitioner.class" aka ProducerConfig.PARTITIONER_CLASS_CONFIG), and applications that use the Kafka’s Streams API must use the same StreamPartitioner for operations such as KStream#to(). The good news is that, if you happen to use the default partitioner-related settings across all @@ -1934,7 +2041,7 @@

      Overview
      KTable-KTable Join

      KTable-KTable joins are always non-windowed joins. They are designed to be consistent with their counterparts in relational databases. The changelog streams of both KTables are materialized into local state stores to represent the - latest snapshot of their table duals. + latest snapshot of their table duals. The join result is a new KTable that represents the changelog stream of the join operation.

      Join output records are effectively created as follows, leveraging the user-supplied ValueJoiner:

      KeyValue<K, LV> leftRecord = ...;
      @@ -2499,13 +2606,13 @@ 

      Overview
      KStream-GlobalKTable Join

      KStream-GlobalKTable joins are always non-windowed joins. They allow you to perform table lookups against a - GlobalKTable (entire changelog stream) upon receiving a new record from the + GlobalKTable (entire changelog stream) upon receiving a new record from the KStream (record stream). An example use case would be “star queries” or “star joins”, where you would enrich a stream of user activities (KStream) with the latest user profile information (GlobalKTable) and further context information (further GlobalKTables).

      At a high-level, KStream-GlobalKTable joins are very similar to KStream-KTable joins. However, global tables provide you - with much more flexibility at the some expense when compared to partitioned + with much more flexibility at the some expense when compared to partitioned tables:

      • They do not require data co-partitioning.
      • @@ -2671,7 +2778,7 @@

        Overviewwindow retention period. + window retention period. Kafka Streams guarantees to keep a window for at least this specified time; the default value is one day and can be changed via Windows#until() and SessionWindows#until().

        The DSL supports the following types of windows:

        @@ -2851,7 +2958,7 @@

        Overviewrecord’s metadata such as its topic, partition, and offset information. + record’s metadata such as its topic, partition, and offset information. However, you don’t want to switch completely to the Processor API just because of that.
      • Migrating from other tools: You are migrating from other stream processing technologies that provide an imperative API, and migrating some of your legacy code to the Processor API was faster and/or easier than to @@ -2877,7 +2984,7 @@

        Overviewprocess() allows you to leverage the Processor API from the DSL. (details)

        This is essentially equivalent to adding the Processor via Topology#addProcessor() to your - processor topology.

        + processor topology.

        An example is available in the javadocs.

    +
    diff --git a/docs/streams/quickstart.html b/docs/streams/quickstart.html index 98871bf181cdb..5cbaf4d8e7118 100644 --- a/docs/streams/quickstart.html +++ b/docs/streams/quickstart.html @@ -165,7 +165,7 @@

    Step 3 Topic:streams-plaintext-input PartitionCount:1 ReplicationFactor:1 Configs: Topic: streams-plaintext-input Partition: 0 Leader: 0 Replicas: 0 Isr: 0 -Topic:streams-wordcount-output PartitionCount:1 ReplicationFactor:1 Configs: +Topic:streams-wordcount-output PartitionCount:1 ReplicationFactor:1 Configs:cleanup.policy=compact Topic: streams-wordcount-output Partition: 0 Leader: 0 Replicas: 0 Isr: 0 @@ -372,10 +372,10 @@

    Step 6: Tear var $navbar = $(".sub-nav-sticky"), y_pos = $navbar.offset().top, height = $navbar.height(); - + $(window).scroll(function() { var scrollTop = $(window).scrollTop(); - + if (scrollTop > y_pos - height) { $navbar.addClass("navbar-fixed") } else if (scrollTop <= y_pos) { diff --git a/docs/streams/upgrade-guide.html b/docs/streams/upgrade-guide.html index 07f8544642458..5ec41036262be 100644 --- a/docs/streams/upgrade-guide.html +++ b/docs/streams/upgrade-guide.html @@ -34,7 +34,7 @@

    Upgrade Guide and API Changes

    - Upgrading from any older version to 2.0.0 is possible: (1) you need to make sure to update you code accordingly, because there are some minor non-compatible API changes since older + Upgrading from any older version to 2.0.0 is possible: (1) you need to make sure to update you code and config accordingly, because there are some minor non-compatible API changes since older releases (the code changes are expected to be minimal, please see below for the details), (2) upgrading to 2.0.0 in the online mode requires two rolling bounces. For (2), in the first rolling bounce phase users need to set config upgrade.from="older version" (possible values are "0.10.0", "0.10.1", "0.10.2", "0.11.0", "1.0", and "1.1") @@ -59,6 +59,16 @@

    Upgrade Guide and API Changes

    For Kafka Streams 0.10.0, broker version 0.10.0 or higher is required.

    +

    + Another important thing to keep in mind: in deprecated KStreamBuilder class, when a KTable is created from a source topic via KStreamBuilder.table(), its materialized state store + will reuse the source topic as its changelog topic for restoring, and will disable logging to avoid appending new updates to the source topic; in the StreamsBuilder class introduced in 1.0, this behavior was changed + accidentally: we still reuse the source topic as the changelog topic for restoring, but will also create a separate changelog topic to append the update records from source topic to. In the 2.0 release, we have fixed this issue and now users + can choose whether or not to reuse the source topic based on the StreamsConfig#TOPOLOGY_OPTIMIZATION: if you are upgrading from the old KStreamBuilder class and hence you need to change your code to use + the new StreamsBuilder, you should set this config value to StreamsConfig#OPTIMIZE to continue reusing the source topic; if you are upgrading from 1.0 or 1.1 where you are already using StreamsBuilder and hence have already + created a separate changelog topic, you should set this config value to StreamsConfig#NO_OPTIMIZATION when upgrading to 2.0.0 in order to use that changelog topic for restoring the state store. + More details about the new config StreamsConfig#TOPOLOGY_OPTIMIZATION can be found in KIP-295. +

    +

    In 2.0.0 we have added a few new APIs on the ReadOnlyWindowStore interface (for details please read Streams API changes below). If you have customized window store implementations that extends the ReadOnlyWindowStore interface you need to make code changes. @@ -113,6 +123,16 @@

    Streams API
  39. Fetched records having a negative timestamp.
  40. +

    + We've also fixed the metrics name for time and session windowed store operations in 2.0. As a result, our current built-in stores + will have their store types in the metric names as in-memory-state, in-memory-lru-state, + rocksdb-state, rocksdb-window-state, and rocksdb-session-state. For example, a RocksDB time windowed store's + put operation metrics would now be + kafka.streams:type=stream-rocksdb-window-state-metrics,client-id=([-.\w]+),task-id=([-.\w]+),rocksdb-window-state-id=([-.\w]+). + Users need to update their metrics collecting and reporting systems for their time and session windowed stores accordingly. + For more details, please read the State Store Metrics section. +

    +

    We have added support for methods in ReadOnlyWindowStore which allows for querying a single window's key-value pair. For users who have customized window store implementations on the above interface, they'd need to update their code to implement the newly added method as well. @@ -124,6 +144,12 @@

    Streams API to let users specify inner serdes if the default serde classes are windowed serdes. For more details, see KIP-265.

    +

    + We've added message header support in the Processor API in Kafka 2.0.0. In particular, we have added a new API ProcessorContext#headers() + which returns a Headers object that keeps track of the headers of the source topic's message that is being processed. Through this object, users can manipulate + the headers map that is being propagated throughout the processor topology as well. For more details please feel free to read + the Developer Guide section. +

    We have deprecated constructors of KafkaStreams that take a StreamsConfig as parameter. Please use the other corresponding constructors that accept java.util.Properties instead. @@ -153,7 +179,8 @@

    Streams API

    Kafka Streams DSL for Scala is a new Kafka Streams client library available for developers authoring Kafka Streams applications in Scala. It wraps core Kafka Streams DSL types to make it easier to call when - interoperating with Scala code. For example, it includes higher order functions as parameters for transformations avoiding the need anonymous classes in Java 7 or experimental SAM type conversions in Scala 2.11, automatic conversion between Java and Scala collection types, a way + interoperating with Scala code. For example, it includes higher order functions as parameters for transformations avoiding the need anonymous classes in Java 7 or experimental SAM type conversions in Scala 2.11, + automatic conversion between Java and Scala collection types, a way to implicitly provide SerDes to reduce boilerplate from your application and make it more typesafe, and more! For more information see the Kafka Streams DSL for Scala documentation and KIP-270. @@ -171,7 +198,7 @@

    Streams API For detailed guidance on how to update your code please read here
  41. KStream, KTable, KGroupedStream overloaded functions that requires serde and other specifications explicitly are removed and replaced with simpler overloaded functions that use Consumed, Produced, Serialized, Materialized, Joined (they are deprecated since 1.0.0). For detailed guidance on how to update your code please read here
  42. -
  43. Processor#punctuate, ValueTransformer#punctuate, ValueTransformer#punctuate and RecordContext#schedule(long) are removed and replaced by RecordContext#schedule(long, PunctuationType, Punctuator) (they are deprecated in 1.0.0).
  44. +
  45. Processor#punctuate, ValueTransformer#punctuate, ValueTransformer#punctuate and ProcessorContext#schedule(long) are removed and replaced by ProcessorContext#schedule(long, PunctuationType, Punctuator) (they are deprecated in 1.0.0).
  46. The second boolean typed parameter "loggingEnabled" in ProcessorContext#register has been removed; users can now use StoreBuilder#withLoggingEnabled, withLoggingDisabled to specify the behavior when they create the state store.
  47. KTable#writeAs, print, foreach, to, through are removed, users can call KTable#tostream()#writeAs instead for the same purpose (they are deprecated since 0.11.0.0). For detailed list of removed APIs please read here
  48. diff --git a/docs/upgrade.html b/docs/upgrade.html index 0430b43eb30ec..6fd0d2cc56c6f 100644 --- a/docs/upgrade.html +++ b/docs/upgrade.html @@ -32,14 +32,14 @@

    Upgrading from 0.8.x, 0.9.x, 0.1 overridden the message format version, you should keep its current value. Alternatively, if you are upgrading from a version prior to 0.11.0.x, then CURRENT_MESSAGE_FORMAT_VERSION should be set to match CURRENT_KAFKA_VERSION. - If you are upgrading from 0.11.0.x, 1.0.x, 1.1.x, or 1.2.x and you have not overridden the message format, then you only need to override + If you are upgrading from 0.11.0.x, 1.0.x, or 1.1.x and you have not overridden the message format, then you only need to override the inter-broker protocol format.
      -
    • inter.broker.protocol.version=CURRENT_KAFKA_VERSION (0.11.0, 1.0, 1.1, 1.2).
    • +
    • inter.broker.protocol.version=CURRENT_KAFKA_VERSION (0.11.0, 1.0, 1.1).
  49. Upgrade the brokers one at a time: shut down the broker, update the code, and restart it.
  50. @@ -64,7 +64,7 @@

    Upgrading from 0.8.x, 0.9.x, 0.1
  51. ACLs should not be added to prefixed resources, (added in KIP-290), until all brokers in the cluster have been updated. -

    NOTE: any prefixed ACLs added to a cluster will be ignored should the cluster be downgraded again. +

    NOTE: any prefixed ACLs added to a cluster, even after the cluster is fully upgraded, will be ignored should the cluster be downgraded again.

  52. @@ -80,10 +80,16 @@

    Notable changes in 2 JMX monitoring tools that do not automatically aggregate. To get the total count for a specific request type, the tool needs to be updated to aggregate across different versions. +
  53. KIP-225 changed the metric "records.lag" to use tags for topic and partition. The original version with the name format "{topic}-{partition}.records-lag" has been removed.
  54. +
  55. The Scala consumers, which have been deprecated since 0.11.0.0, have been removed. The Java consumer has been the recommended option + since 0.10.0.0. Note that the Scala consumers in 1.1.0 (and older) will continue to work even if the brokers are upgraded to 2.0.0.
  56. The Scala producers, which have been deprecated since 0.10.0.0, have been removed. The Java producer has been the recommended option since 0.9.0.0. Note that the behaviour of the default partitioner in the Java producer differs from the default partitioner - in the Scala producers. Users migrating should consider configuring a custom partitioner that retains the previous behaviour.
  57. -
  58. The ConsoleProducer no longer supports the Scala producer.
  59. + in the Scala producers. Users migrating should consider configuring a custom partitioner that retains the previous behaviour. + Note that the Scala producers in 1.1.0 (and older) will continue to work even if the brokers are upgraded to 2.0.0. +
  60. MirrorMaker and ConsoleConsumer no longer support the Scala consumer, they always use the Java consumer.
  61. +
  62. The ConsoleProducer no longer supports the Scala producer, it always uses the Java producer.
  63. +
  64. A number of deprecated tools that rely on the Scala clients have been removed: ReplayLogProducer, SimpleConsumerPerformance, SimpleConsumerShell, ExportZkOffsets, ImportZkOffsets, UpdateOffsetsInZK, VerifyConsumerRebalance.
  65. The deprecated kafka.tools.ProducerPerformance has been removed, please use org.apache.kafka.tools.ProducerPerformance.
  66. New Kafka Streams configuration parameter upgrade.from added that allows rolling bounce upgrade from older version.
  67. KIP-284 changed the retention time for Kafka Streams repartition topics by setting its default value to Long.MAX_VALUE.
  68. @@ -97,12 +103,19 @@
    Notable changes in 2 internal.value.converter=org.apache.kafka.connect.json.JsonConverter internal.value.converter.schemas.enable=false -
  69. KIP-266 adds overloads to the consumer to support - timeout behavior for blocking APIs. In particular, a new poll(Duration) API has been added which - does not block for dynamic partition assignment. The old poll(long) API has been deprecated and - will be removed in a future version.
  70. +
  71. KIP-266 adds a new consumer configuration default.api.timeout.ms + to specify the default timeout to use for KafkaConsumer APIs that could block. The KIP also adds overloads for such blocking + APIs to support specifying a specific timeout to use for each of them instead of using the default timeout set by default.api.timeout.ms. + In particular, a new poll(Duration) API has been added which does not block for dynamic partition assignment. + The old poll(long) API has been deprecated and will be removed in a future version. Overloads have also been added + for other KafkaConsumer methods like partitionsFor, listTopics, offsetsForTimes, + beginningOffsets, endOffsets and close that take in a Duration.
  72. +
  73. Also as part of KIP-266, the default value of request.timeout.ms has been changed to 30 seconds. + The previous value was a little higher than 5 minutes to account for maximum time that a rebalance would take. + Now we treat the JoinGroup request in the rebalance as a special case and use a value derived from + max.poll.interval.ms for the request timeout. All other request types use the timeout defined + by request.timeout.ms
  74. The internal method kafka.admin.AdminClient.deleteRecordsBefore has been removed. Users are encouraged to migrate to org.apache.kafka.clients.admin.AdminClient.deleteRecords.
  75. -
  76. The tool kafka.tools.ReplayLogProducer has been removed.
  77. The AclCommand tool --producer convenience option uses the KIP-277 finer grained ACL on the given topic.
  78. KIP-176 removes the --new-consumer option for all consumer based tools. This option is redundant since the new consumer is automatically @@ -110,6 +123,19 @@
    Notable changes in 2
  79. KIP-290 adds the ability to define ACLs on prefixed resources, e.g. any topic starting with 'foo'.
  80. +
  81. KIP-283 improves message down-conversion + handling on Kafka broker, which has typically been a memory-intensive operation. The KIP adds a mechanism by which the operation becomes less memory intensive + by down-converting chunks of partition data at a time which helps put an upper bound on memory consumption. With this improvement, there is a change in + FetchResponse protocol behavior where the broker could send an oversized message batch towards the end of the response with an invalid offset. + Such oversized messages must be ignored by consumer clients, as is done by KafkaConsumer. +

    KIP-283 also adds new topic and broker configurations message.downconversion.enable and log.message.downconversion.enable respectively + to control whether down-conversion is enabled. When disabled, broker does not perform any down-conversion and instead sends an UNSUPPORTED_VERSION + error to the client.

  82. +
  83. Dynamic broker configuration options can be stored in ZooKeeper using kafka-configs.sh before brokers are started. + This option can be used to avoid storing clear passwords in server.properties as all password configs may be stored encrypted in ZooKeeper.
  84. +
  85. ZooKeeper hosts are now re-resolved if connection attempt fails. But if your ZooKeeper host names resolve + to multiple addresses and some of them are not reachable, then you may need to increase the connection timeout + zookeeper.connection.timeout.ms.
  86. New Protocol Versions
    @@ -120,10 +146,10 @@

    New Prot -
    Upgrading a 2.0.0 Kafka Streams Application
    +
    Upgrading a 1.1 Kafka Streams Application
      -
    • Upgrading your Streams application from 1.1.0 to 2.0.0 does not require a broker upgrade. - A Kafka Streams 2.0.0 application can connect to 2.0, 1.1, 1.0, 0.11.0, 0.10.2 and 0.10.1 brokers (it is not possible to connect to 0.10.0 brokers though).
    • +
    • Upgrading your Streams application from 1.1 to 2.0 does not require a broker upgrade. + A Kafka Streams 2.0 application can connect to 2.0, 1.1, 1.0, 0.11.0, 0.10.2 and 0.10.1 brokers (it is not possible to connect to 0.10.0 brokers though).
    • Note that in 2.0 we have removed the public APIs that are deprecated prior to 1.0; users leveraging on those deprecated APIs need to make code changes accordingly. See Streams API changes in 2.0.0 for more details.
    @@ -172,14 +198,11 @@

    Upgrading from 0.8.x, 0.9.x, 0.1 Hot-swapping the jar-file only might not work. - -
    Notable changes in 1.1.0
    -

    Upgrading a 1.1.0 Kafka Streams Application
    +
    Upgrading a 1.0 Kafka Streams Application
      -
    • Upgrading your Streams application from 1.0.0 to 1.1.0 does not require a broker upgrade. - A Kafka Streams 1.1.0 application can connect to 1.0, 0.11.0, 0.10.2 and 0.10.1 brokers (it is not possible to connect to 0.10.0 brokers though).
    • +
    • Upgrading your Streams application from 1.0 to 1.1 does not require a broker upgrade. + A Kafka Streams 1.1 application can connect to 1.0, 0.11.0, 0.10.2 and 0.10.1 brokers (it is not possible to connect to 0.10.0 brokers though).
    • See Streams API changes in 1.1.0 for more details.
    @@ -255,13 +278,11 @@

    Upgrading from 0.8.x, 0.9.x, 0.1 Similarly for the message format version. -
    Notable changes in 1.0.1
    -
    Upgrading a 0.11.0 Kafka Streams Application
    +

    Upgrading a 0.11.0 Kafka Streams Application
      -
    • Upgrading your Streams application from 0.11.0 to 1.0.0 does not require a broker upgrade. - A Kafka Streams 1.0.0 application can connect to 0.11.0, 0.10.2 and 0.10.1 brokers (it is not possible to connect to 0.10.0 brokers though). +
    • Upgrading your Streams application from 0.11.0 to 1.0 does not require a broker upgrade. + A Kafka Streams 1.0 application can connect to 0.11.0, 0.10.2 and 0.10.1 brokers (it is not possible to connect to 0.10.0 brokers though). However, Kafka Streams 1.0 requires 0.10 message format or newer and does not work with older message formats.
    • If you are monitoring on streams metrics, you will need make some changes to the metrics names in your reporting and monitoring code, because the metrics sensor hierarchy was changed.
    • There are a few public APIs including ProcessorContext#schedule(), Processor#punctuate() and KStreamBuilder, TopologyBuilder are being deprecated by new APIs. @@ -374,7 +395,6 @@
      Streams API changes in 0.10.2, and Streams API changes in 0.10.1 for more details). Thus, you need to update and recompile your code. Just swapping the Kafka Streams library jar file will not work and will break your application.
    • -
    • Upgrading from 0.10.0.x to 1.0.0 or 1.0.1 requires an offline upgrade (rolling bounce upgrade is not supported)
        @@ -441,7 +460,7 @@

        Upgrading from 0.8.x, 0.9.x, 0 before you switch to 0.11.0. -
        Upgrading a 0.10.2 Kafka Streams Application
        +

        Upgrading a 0.10.2 Kafka Streams Application
        • Upgrading your Streams application from 0.10.2 to 0.11.0 does not require a broker upgrade. A Kafka Streams 0.11.0 application can connect to 0.11.0, 0.10.2 and 0.10.1 brokers (it is not possible to connect to 0.10.0 brokers though).
        • @@ -468,7 +487,6 @@
          Streams API changes in 0.10.2, and Streams API changes in 0.10.1 for more details). Thus, you need to update and recompile your code. Just swapping the Kafka Streams library jar file will not work and will break your application. -
        • Upgrading from 0.10.0.x to 0.11.0.0, 0.11.0.1, or 0.11.0.2 requires an offline upgrade (rolling bounce upgrade is not supported)
          • stop all old (0.10.0.x) application instances
          • @@ -489,13 +506,11 @@
            Notable changes in 0.11.0.3
            • New Kafka Streams configuration parameter upgrade.from added that allows rolling bounce upgrade from version 0.10.0.x
            • See the Kafka Streams upgrade guide for details about this new config.
            --->
            Notable changes in 0.11.0.0
              @@ -637,7 +652,7 @@

              Upgrading from 0.8.x, 0.9.x, 0

              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 a 0.10.1 Kafka Streams Application
              +

              Upgrading a 0.10.1 Kafka Streams Application
              • Upgrading your Streams application from 0.10.1 to 0.10.2 does not require a broker upgrade. A Kafka Streams 0.10.2 application can connect to 0.10.2 and 0.10.1 brokers (it is not possible to connect to 0.10.0 brokers though).
              • @@ -652,7 +667,6 @@
                broker upgrade because a Kafka Streams 0.10.2 application can only connect to 0.10.2 or 0.10.1 brokers.
              • There are couple of API changes, that are not backward compatible (cf. Streams API changes in 0.10.2 for more details). Thus, you need to update and recompile your code. Just swapping the Kafka Streams library jar file will not work and will break your application.
              • -
              • Upgrading from 0.10.0.x to 0.10.2.0 or 0.10.2.1 requires an offline upgrade (rolling bounce upgrade is not supported)
                • stop all old (0.10.0.x) application instances
                • @@ -673,12 +686,10 @@
                  Notable changes in 0.10.2.2
                  • New configuration parameter upgrade.from added that allows rolling bounce upgrade from version 0.10.0.x
                  --->
                  Notable changes in 0.10.2.1
                    @@ -744,6 +755,13 @@

                    Upgrading from 0.8.x, 0.9.x or 0.1

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

                    Potential breaking changes in 0.10.1.0
                    -
                    Upgrading a 0.10.0 Kafka Streams Application
                    +

                    Upgrading a 0.10.0 Kafka Streams Application
                    • Upgrading your Streams application from 0.10.0 to 0.10.1 does require a broker upgrade because a Kafka Streams 0.10.1 application can only connect to 0.10.1 brokers.
                    • There are couple of API changes, that are not backward compatible (cf. Streams API changes in 0.10.1 for more details). - Thus, you need to update and recompile your code. Just swapping the Kafka Streams library jar file will not work and will break your application.
                    • - +
                        +
                      • prepare your application instances for a rolling bounce and make sure that config upgrade.from is set to "0.10.0" for new version 0.10.1.2
                      • +
                      • bounce each instance of your application once
                      • +
                      • prepare your newly deployed 0.10.1.2 application instances for a second round of rolling bounces; make sure to remove the value for config upgrade.mode
                      • +
                      • bounce each instance of your application once more to complete the upgrade
                      • +
                      +
                    • Upgrading from 0.10.0.x to 0.10.1.0 or 0.10.1.1 requires an offline upgrade (rolling bounce upgrade is not supported)
                      • stop all old (0.10.0.x) application instances
                      • diff --git a/examples/src/main/java/kafka/examples/SimpleConsumerDemo.java b/examples/src/main/java/kafka/examples/SimpleConsumerDemo.java deleted file mode 100644 index 6ebd7910d9bbb..0000000000000 --- a/examples/src/main/java/kafka/examples/SimpleConsumerDemo.java +++ /dev/null @@ -1,96 +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.examples; - -import kafka.api.FetchRequest; -import kafka.api.FetchRequestBuilder; -import kafka.javaapi.FetchResponse; -import kafka.javaapi.consumer.SimpleConsumer; -import kafka.javaapi.message.ByteBufferMessageSet; -import kafka.message.MessageAndOffset; - -import java.io.UnsupportedEncodingException; -import java.nio.ByteBuffer; -import java.util.Collections; -import java.util.HashMap; -import java.util.List; -import java.util.Map; - -/** - * @deprecated since 0.11.0.0. This class will be removed in a future release. - */ -@Deprecated -public class SimpleConsumerDemo { - - private static void printMessages(ByteBufferMessageSet messageSet) throws UnsupportedEncodingException { - for (MessageAndOffset messageAndOffset : messageSet) { - ByteBuffer payload = messageAndOffset.message().payload(); - byte[] bytes = new byte[payload.limit()]; - payload.get(bytes); - System.out.println(new String(bytes, "UTF-8")); - } - } - - private static void generateData() { - Producer producer2 = new Producer(KafkaProperties.TOPIC2, false); - producer2.start(); - Producer producer3 = new Producer(KafkaProperties.TOPIC3, false); - producer3.start(); - try { - Thread.sleep(1000); - } catch (InterruptedException e) { - e.printStackTrace(); - } - } - - public static void main(String[] args) throws Exception { - generateData(); - - SimpleConsumer simpleConsumer = new SimpleConsumer(KafkaProperties.KAFKA_SERVER_URL, - KafkaProperties.KAFKA_SERVER_PORT, - KafkaProperties.CONNECTION_TIMEOUT, - KafkaProperties.KAFKA_PRODUCER_BUFFER_SIZE, - KafkaProperties.CLIENT_ID); - - System.out.println("Testing single fetch"); - FetchRequest req = new FetchRequestBuilder() - .clientId(KafkaProperties.CLIENT_ID) - .addFetch(KafkaProperties.TOPIC2, 0, 0L, 100) - .build(); - FetchResponse fetchResponse = simpleConsumer.fetch(req); - printMessages(fetchResponse.messageSet(KafkaProperties.TOPIC2, 0)); - - System.out.println("Testing single multi-fetch"); - Map> topicMap = new HashMap<>(); - topicMap.put(KafkaProperties.TOPIC2, Collections.singletonList(0)); - topicMap.put(KafkaProperties.TOPIC3, Collections.singletonList(0)); - req = new FetchRequestBuilder() - .clientId(KafkaProperties.CLIENT_ID) - .addFetch(KafkaProperties.TOPIC2, 0, 0L, 100) - .addFetch(KafkaProperties.TOPIC3, 0, 0L, 100) - .build(); - fetchResponse = simpleConsumer.fetch(req); - int fetchReq = 0; - for (Map.Entry> entry : topicMap.entrySet()) { - String topic = entry.getKey(); - for (Integer offset : entry.getValue()) { - System.out.println("Response from fetch request no: " + ++fetchReq); - printMessages(fetchResponse.messageSet(topic, offset)); - } - } - } -} diff --git a/gradle.properties b/gradle.properties index 3fec74a02647f..a2345757d3685 100644 --- a/gradle.properties +++ b/gradle.properties @@ -16,7 +16,7 @@ group=org.apache.kafka # NOTE: When you change this version number, you should also make sure to update # the version numbers in tests/kafkatest/__init__.py and kafka-merge-pr.py. -version=2.0.0-SNAPSHOT +version=2.0.2-SNAPSHOT scalaVersion=2.11.12 task=build org.gradle.jvmargs=-Xmx1024m -Xss2m diff --git a/gradle/dependencies.gradle b/gradle/dependencies.gradle index ce2db721025d8..d3f70d14a2c5b 100644 --- a/gradle/dependencies.gradle +++ b/gradle/dependencies.gradle @@ -52,23 +52,25 @@ versions += [ apacheds: "2.0.0-M24", argparse4j: "0.7.0", bcpkix: "1.59", + httpclient: "4.5.7", easymock: "3.6", - jackson: "2.9.5", - jetty: "9.4.10.v20180503", + jackson: "2.9.7", + jetty: "9.4.11.v20180605", jersey: "2.27", jmh: "1.21", log4j: "1.2.17", scalaLogging: "3.9.0", jaxb: "2.3.0", + jaxrs: "2.1", jfreechart: "1.0.0", jopt: "5.0.4", junit: "4.12", kafka_0100: "0.10.0.1", kafka_0101: "0.10.1.1", - kafka_0102: "0.10.2.1", - kafka_0110: "0.11.0.2", - kafka_10: "1.0.1", - kafka_11: "1.1.0", + kafka_0102: "0.10.2.2", + kafka_0110: "0.11.0.3", + kafka_10: "1.0.2", + kafka_11: "1.1.1", lz4: "1.4.1", mavenArtifact: "3.5.3", metrics: "2.2.0", @@ -81,7 +83,7 @@ versions += [ slf4j: "1.7.25", snappy: "1.1.7.1", zkclient: "0.10", - zookeeper: "3.4.12" + zookeeper: "3.4.13" ] libs += [ @@ -101,6 +103,7 @@ libs += [ jacksonDatabind: "com.fasterxml.jackson.core:jackson-databind:$versions.jackson", jacksonJaxrsJsonProvider: "com.fasterxml.jackson.jaxrs:jackson-jaxrs-json-provider:$versions.jackson", jaxbApi: "javax.xml.bind:jaxb-api:$versions.jaxb", + jaxrsApi: "javax.ws.rs:javax.ws.rs-api:$versions.jaxrs", jettyServer: "org.eclipse.jetty:jetty-server:$versions.jetty", jettyClient: "org.eclipse.jetty:jetty-client:$versions.jetty", jettyServlet: "org.eclipse.jetty:jetty-servlet:$versions.jetty", @@ -138,5 +141,6 @@ libs += [ zkclient: "com.101tec:zkclient:$versions.zkclient", zookeeper: "org.apache.zookeeper:zookeeper:$versions.zookeeper", jfreechart: "jfreechart:jfreechart:$versions.jfreechart", - mavenArtifact: "org.apache.maven:maven-artifact:$versions.mavenArtifact" + mavenArtifact: "org.apache.maven:maven-artifact:$versions.mavenArtifact", + httpclient: "org.apache.httpcomponents:httpclient:$versions.httpclient" ] diff --git a/gradle/findbugs-exclude.xml b/gradle/findbugs-exclude.xml index 70c9e770ec3b3..21c889116d75d 100644 --- a/gradle/findbugs-exclude.xml +++ b/gradle/findbugs-exclude.xml @@ -101,13 +101,6 @@ For a detailed description of findbugs bug categories, see http://findbugs.sourc - - - - - - - @@ -269,14 +262,6 @@ For a detailed description of findbugs bug categories, see http://findbugs.sourc - - - - - - - diff --git a/gradle/wrapper/gradle-wrapper.properties b/gradle/wrapper/gradle-wrapper.properties new file mode 100644 index 0000000000000..3de9b8daae54a --- /dev/null +++ b/gradle/wrapper/gradle-wrapper.properties @@ -0,0 +1,19 @@ +# +# Copyright 2017 the original author or authors. +# +# 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. +distributionBase=GRADLE_USER_HOME +distributionPath=wrapper/dists +distributionUrl=https\://services.gradle.org/distributions/gradle-4.8.1-all.zip +zipStoreBase=GRADLE_USER_HOME +zipStorePath=wrapper/dists diff --git a/gradlew b/gradlew new file mode 100755 index 0000000000000..b45bdfd7f989f --- /dev/null +++ b/gradlew @@ -0,0 +1,215 @@ +#!/usr/bin/env sh +# +# Copyright 2017 the original author or authors. +# +# 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. + +# +# Copyright 2015 the original author or authors. +# +# 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 +# +# https://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# + +############################################################################## +## +## Gradle start up script for UN*X +## +############################################################################## + +# Attempt to set APP_HOME +# Resolve links: $0 may be a link +PRG="$0" +# Need this for relative symlinks. +while [ -h "$PRG" ] ; do + ls=`ls -ld "$PRG"` + link=`expr "$ls" : '.*-> \(.*\)$'` + if expr "$link" : '/.*' > /dev/null; then + PRG="$link" + else + PRG=`dirname "$PRG"`"/$link" + fi +done +SAVED="`pwd`" +cd "`dirname \"$PRG\"`/" >/dev/null +APP_HOME="`pwd -P`" +cd "$SAVED" >/dev/null + +APP_NAME="Gradle" +APP_BASE_NAME=`basename "$0"` + +# Add default JVM options here. You can also use JAVA_OPTS and GRADLE_OPTS to pass JVM options to this script. +DEFAULT_JVM_OPTS='"-Xmx64m" "-Xms64m"' + +# Use the maximum available, or set MAX_FD != -1 to use that value. +MAX_FD="maximum" + +warn () { + echo "$*" +} + +die () { + echo + echo "$*" + echo + exit 1 +} + +# OS specific support (must be 'true' or 'false'). +cygwin=false +msys=false +darwin=false +nonstop=false +case "`uname`" in + CYGWIN* ) + cygwin=true + ;; + Darwin* ) + darwin=true + ;; + MINGW* ) + msys=true + ;; + NONSTOP* ) + nonstop=true + ;; +esac + + +# Loop in case we encounter an error. +for attempt in 1 2 3; do + if [ ! -e $APP_HOME/gradle/wrapper/gradle-wrapper.jar ]; then + if ! curl -s -S --retry 3 -L -o "$APP_HOME/gradle/wrapper/gradle-wrapper.jar" "https://raw.githubusercontent.com/gradle/gradle/vnull/gradle/wrapper/gradle-wrapper.jar"; then + rm -f "$APP_HOME/gradle/wrapper/gradle-wrapper.jar" + # Pause for a bit before looping in case the server throttled us. + sleep 5 + continue + fi + fi +done + +CLASSPATH=$APP_HOME/gradle/wrapper/gradle-wrapper.jar + +# Determine the Java command to use to start the JVM. +if [ -n "$JAVA_HOME" ] ; then + if [ -x "$JAVA_HOME/jre/sh/java" ] ; then + # IBM's JDK on AIX uses strange locations for the executables + JAVACMD="$JAVA_HOME/jre/sh/java" + else + JAVACMD="$JAVA_HOME/bin/java" + fi + if [ ! -x "$JAVACMD" ] ; then + die "ERROR: JAVA_HOME is set to an invalid directory: $JAVA_HOME + +Please set the JAVA_HOME variable in your environment to match the +location of your Java installation." + fi +else + JAVACMD="java" + which java >/dev/null 2>&1 || die "ERROR: JAVA_HOME is not set and no 'java' command could be found in your PATH. + +Please set the JAVA_HOME variable in your environment to match the +location of your Java installation." +fi + +# Increase the maximum file descriptors if we can. +if [ "$cygwin" = "false" -a "$darwin" = "false" -a "$nonstop" = "false" ] ; then + MAX_FD_LIMIT=`ulimit -H -n` + if [ $? -eq 0 ] ; then + if [ "$MAX_FD" = "maximum" -o "$MAX_FD" = "max" ] ; then + MAX_FD="$MAX_FD_LIMIT" + fi + ulimit -n $MAX_FD + if [ $? -ne 0 ] ; then + warn "Could not set maximum file descriptor limit: $MAX_FD" + fi + else + warn "Could not query maximum file descriptor limit: $MAX_FD_LIMIT" + fi +fi + +# For Darwin, add options to specify how the application appears in the dock +if $darwin; then + GRADLE_OPTS="$GRADLE_OPTS \"-Xdock:name=$APP_NAME\" \"-Xdock:icon=$APP_HOME/media/gradle.icns\"" +fi + +# For Cygwin or MSYS, switch paths to Windows format before running java +if [ "$cygwin" = "true" -o "$msys" = "true" ] ; then + APP_HOME=`cygpath --path --mixed "$APP_HOME"` + CLASSPATH=`cygpath --path --mixed "$CLASSPATH"` + JAVACMD=`cygpath --unix "$JAVACMD"` + + # We build the pattern for arguments to be converted via cygpath + ROOTDIRSRAW=`find -L / -maxdepth 1 -mindepth 1 -type d 2>/dev/null` + SEP="" + for dir in $ROOTDIRSRAW ; do + ROOTDIRS="$ROOTDIRS$SEP$dir" + SEP="|" + done + OURCYGPATTERN="(^($ROOTDIRS))" + # Add a user-defined pattern to the cygpath arguments + if [ "$GRADLE_CYGPATTERN" != "" ] ; then + OURCYGPATTERN="$OURCYGPATTERN|($GRADLE_CYGPATTERN)" + fi + # Now convert the arguments - kludge to limit ourselves to /bin/sh + i=0 + for arg in "$@" ; do + CHECK=`echo "$arg"|egrep -c "$OURCYGPATTERN" -` + CHECK2=`echo "$arg"|egrep -c "^-"` ### Determine if an option + + if [ $CHECK -ne 0 ] && [ $CHECK2 -eq 0 ] ; then ### Added a condition + eval `echo args$i`=`cygpath --path --ignore --mixed "$arg"` + else + eval `echo args$i`="\"$arg\"" + fi + i=$((i+1)) + done + case $i in + (0) set -- ;; + (1) set -- "$args0" ;; + (2) set -- "$args0" "$args1" ;; + (3) set -- "$args0" "$args1" "$args2" ;; + (4) set -- "$args0" "$args1" "$args2" "$args3" ;; + (5) set -- "$args0" "$args1" "$args2" "$args3" "$args4" ;; + (6) set -- "$args0" "$args1" "$args2" "$args3" "$args4" "$args5" ;; + (7) set -- "$args0" "$args1" "$args2" "$args3" "$args4" "$args5" "$args6" ;; + (8) set -- "$args0" "$args1" "$args2" "$args3" "$args4" "$args5" "$args6" "$args7" ;; + (9) set -- "$args0" "$args1" "$args2" "$args3" "$args4" "$args5" "$args6" "$args7" "$args8" ;; + esac +fi + +# Escape application args +save () { + for i do printf %s\\n "$i" | sed "s/'/'\\\\''/g;1s/^/'/;\$s/\$/' \\\\/" ; done + echo " " +} +APP_ARGS=$(save "$@") + +# Collect all arguments for the java command, following the shell quoting and substitution rules +eval set -- $DEFAULT_JVM_OPTS $JAVA_OPTS $GRADLE_OPTS "\"-Dorg.gradle.appname=$APP_BASE_NAME\"" -classpath "\"$CLASSPATH\"" org.gradle.wrapper.GradleWrapperMain "$APP_ARGS" + +# by default we should be in the correct project dir, but when run from Finder on Mac, the cwd is wrong +if [ "$(uname)" = "Darwin" ] && [ "$HOME" = "$PWD" ]; then + cd "$(dirname "$0")" +fi + +exec "$JAVACMD" "$@" diff --git a/jenkins.sh b/jenkins.sh index f3ec91bf676df..b03195eb5b065 100755 --- a/jenkins.sh +++ b/jenkins.sh @@ -17,4 +17,4 @@ # This script is used for verifying changes in Jenkins. In order to provide faster feedback, the tasks are ordered so # that faster tasks are executed in every module before slower tasks (if possible). For example, the unit tests for all # the modules are executed before the integration tests. -./gradlew clean compileJava compileScala compileTestJava compileTestScala checkstyleMain checkstyleTest findbugsMain unitTest rat integrationTest --no-daemon -PxmlFindBugsReport=true -PtestLoggingEvents=started,passed,skipped,failed "$@" +./gradlew clean compileJava compileScala compileTestJava compileTestScala checkstyleMain checkstyleTest findbugsMain unitTest rat integrationTest --no-daemon --continue -PxmlFindBugsReport=true -PtestLoggingEvents=started,passed,skipped,failed "$@" diff --git a/release.py b/release.py index 7eff6cc0d1bc8..802c9de648003 100755 --- a/release.py +++ b/release.py @@ -45,6 +45,10 @@ With no arguments this script assumes you have the Kafka repository and kafka-site repository checked out side-by-side, but you can specify a full path to the kafka-site repository if this is not the case. +release.py release-email + + Generates the email content/template for sending release announcement email. + """ from __future__ import print_function @@ -56,6 +60,7 @@ import subprocess import sys import tempfile +import re PROJECT_NAME = "kafka" CAPITALIZED_PROJECT_NAME = "kafka".upper() @@ -147,7 +152,7 @@ def sftp_mkdir(dir): try: cmd_str = """ cd %s -mkdir %s +-mkdir %s """ % (basedir, dirname) cmd("Creating '%s' in '%s' in your Apache home directory if it does not exist (errors are ok if the directory already exists)" % (dirname, basedir), "sftp -b - %s@home.apache.org" % apache_id, stdin=cmd_str, allow_failure=True) except subprocess.CalledProcessError: @@ -256,11 +261,138 @@ def command_stage_docs(): sys.exit(0) +def validate_release_version_parts(version): + try: + version_parts = version.split('.') + if len(version_parts) != 3: + fail("Invalid release version, should have 3 version number components") + # Validate each part is a number + [int(x) for x in version_parts] + except ValueError: + fail("Invalid release version, should be a dotted version number") + +def get_release_version_parts(version): + validate_release_version_parts(version) + return version.split('.') + +def validate_release_num(version): + tags = cmd_output('git tag').split() + if version not in tags: + fail("The specified version is not a valid release version number") + validate_release_version_parts(version) + +def command_release_announcement_email(): + tags = cmd_output('git tag').split() + release_tag_pattern = re.compile('^[0-9]+\.[0-9]+\.[0-9]+$') + release_tags = sorted([t for t in tags if re.match(release_tag_pattern, t)]) + release_version_num = release_tags[-1] + if not user_ok("""Is the current release %s ? (y/n): """ % release_version_num): + release_version_num = raw_input('What is the current release version:') + validate_release_num(release_version_num) + previous_release_version_num = release_tags[-2] + if not user_ok("""Is the previous release %s ? (y/n): """ % previous_release_version_num): + previous_release_version_num = raw_input('What is the previous release version:') + validate_release_num(previous_release_version_num) + if release_version_num < previous_release_version_num : + fail("Current release version number can't be less than previous release version number") + number_of_contributors = int(subprocess.check_output('git shortlog -sn --no-merges %s..%s | wc -l' % (previous_release_version_num, release_version_num) , shell=True)) + contributors = subprocess.check_output("git shortlog -sn --no-merges %s..%s | cut -f2 | sort --ignore-case" % (previous_release_version_num, release_version_num), shell=True) + release_announcement_data = { + 'number_of_contributors': number_of_contributors, + 'contributors': ', '.join(str(x) for x in filter(None, contributors.split('\n'))), + 'release_version': release_version_num + } + + release_announcement_email = """ +To: announce@apache.org, dev@kafka.apache.org, users@kafka.apache.org, kafka-clients@googlegroups.com +Subject: [ANNOUNCE] Apache Kafka %(release_version)s + +The Apache Kafka community is pleased to announce the release for Apache Kafka %(release_version)s + +
                        + +All of the changes in this release can be found in the release notes: +https://www.apache.org/dist/kafka/%(release_version)s/RELEASE_NOTES.html + + +You can download the source and binary release (Scala ) from: +https://kafka.apache.org/downloads#%(release_version)s + +--------------------------------------------------------------------------------------------------- + + +Apache Kafka is a distributed streaming platform with four core APIs: + + +** The Producer API allows an application to publish a stream records to +one or more Kafka topics. + +** The Consumer API allows an application to subscribe to one or more +topics and process the stream of records produced to them. + +** The Streams API allows an application to act as a stream processor, +consuming an input stream from one or more topics and producing an +output stream to one or more output topics, effectively transforming the +input streams to output streams. + +** The Connector API allows building and running reusable producers or +consumers that connect Kafka topics to existing applications or data +systems. For example, a connector to a relational database might +capture every change to a table. + + +With these APIs, Kafka can be used for two broad classes of application: + +** Building real-time streaming data pipelines that reliably get data +between systems or applications. + +** Building real-time streaming applications that transform or react +to the streams of data. + + +Apache Kafka is in use at large and small companies worldwide, including +Capital One, Goldman Sachs, ING, LinkedIn, Netflix, Pinterest, Rabobank, +Target, The New York Times, Uber, Yelp, and Zalando, among others. + +A big thank you for the following %(number_of_contributors)d contributors to this release! + +%(contributors)s + +We welcome your help and feedback. For more information on how to +report problems, and to get involved, visit the project website at +https://kafka.apache.org/ + +Thank you! + + +Regards, + +""" % release_announcement_data + + print() + print("*****************************************************************") + print() + print(release_announcement_email) + print() + print("*****************************************************************") + print() + print("Use the above template to send the announcement for the release to the mailing list.") + print("IMPORTANT: Note that there are still some substitutions that need to be made in the template:") + print(" - Describe major changes in this release") + print(" - Scala versions") + print(" - Fill in your name in the signature") + print(" - You will need to use your apache email address to send out the email (otherwise, it won't be delivered to announce@apache.org)") + print(" - Finally, validate all the links before shipping!") + print("Note that all substitutions are annotated with <> around them.") + sys.exit(0) + # Dispatch to subcommand subcommand = sys.argv[1] if len(sys.argv) > 1 else None if subcommand == 'stage-docs': command_stage_docs() +elif subcommand == 'release-email': + command_release_announcement_email() elif not (subcommand is None or subcommand == 'stage'): fail("Unknown subcommand: %s" % subcommand) # else -> default subcommand stage @@ -328,14 +460,7 @@ def command_stage_docs(): cmd("Verifying that you have no staged git changes", 'git diff --cached --exit-code --quiet') release_version = raw_input("Release version (without any RC info, e.g. 1.0.0): ") -try: - release_version_parts = release_version.split('.') - if len(release_version_parts) != 3: - fail("Invalid release version, should have 3 version number components") - # Validate each part is a number - [int(x) for x in release_version_parts] -except ValueError: - fail("Invalid release version, should be a dotted version number") +release_version_parts = get_release_version_parts(release_version) rc = raw_input("Release candidate number: ") @@ -479,23 +604,22 @@ def select_gpg_key(): for root, dirs, files in os.walk(artifacts_dir): assert root.startswith(artifacts_dir) - for file in files: - local_path = os.path.join(root, file) - remote_path = os.path.join("public_html", kafka_output_dir, root[len(artifacts_dir)+1:], file) - sftp_cmds += "\nput %s %s" % (local_path, remote_path) - for dir in dirs: sftp_mkdir(os.path.join("public_html", kafka_output_dir, root[len(artifacts_dir)+1:], dir)) -if sftp_cmds: - cmd("Uploading artifacts in %s to your Apache home directory" % root, "sftp -b - %s@home.apache.org" % apache_id, stdin=sftp_cmds) + for file in files: + local_path = os.path.join(root, file) + remote_path = os.path.join("public_html", kafka_output_dir, root[len(artifacts_dir)+1:], file) + sftp_cmds = """ +put %s %s +""" % (local_path, remote_path) + cmd("Uploading artifacts in %s to your Apache home directory" % root, "sftp -b - %s@home.apache.org" % apache_id, stdin=sftp_cmds) with open(os.path.expanduser("~/.gradle/gradle.properties")) as f: contents = f.read() if not user_ok("Going to build and upload mvn artifacts based on these settings:\n" + contents + '\nOK (y/n)?: '): fail("Retry again later") cmd("Building and uploading archives", "./gradlew uploadArchivesAll", cwd=kafka_dir, env=jdk8_env) -cmd("Building and uploading archives", "./gradlew uploadCoreArchives_2_12 -PscalaVersion=2.12", cwd=kafka_dir, env=jdk8_env) cmd("Building and uploading archives", "mvn deploy -Pgpg-signing", cwd=streams_quickstart_dir, env=jdk8_env) release_notification_props = { 'release_version': release_version, @@ -581,7 +705,7 @@ def select_gpg_key(): http://home.apache.org/~%(apache_id)s/kafka-%(rc_tag)s/javadoc/ * Tag to be voted upon (off %(dev_branch)s branch) is the %(release_version)s tag: -https://git-wip-us.apache.org/repos/asf?p=kafka.git;a=tag;h=%(rc_githash)s +https://github.com/apache/kafka/releases/tag/%(rc_tag)s * Documentation: http://kafka.apache.org/%(docs_version)s/documentation.html @@ -591,7 +715,7 @@ def select_gpg_key(): * Successful Jenkins builds for the %(dev_branch)s branch: Unit/integration tests: https://builds.apache.org/job/kafka-%(dev_branch)s-jdk8// -System tests: https://jenkins.confluent.io/job/system-test-kafka-%(dev_branch)s// +System tests: https://jenkins.confluent.io/job/system-test-kafka/job/%(dev_branch)s// /************************************** diff --git a/streams/examples/src/main/java/org/apache/kafka/streams/examples/pageview/PageViewTypedDemo.java b/streams/examples/src/main/java/org/apache/kafka/streams/examples/pageview/PageViewTypedDemo.java index bd24e849ef5a5..234d3fc7e2ebb 100644 --- a/streams/examples/src/main/java/org/apache/kafka/streams/examples/pageview/PageViewTypedDemo.java +++ b/streams/examples/src/main/java/org/apache/kafka/streams/examples/pageview/PageViewTypedDemo.java @@ -28,16 +28,14 @@ import org.apache.kafka.streams.StreamsConfig; import org.apache.kafka.streams.kstream.KStream; import org.apache.kafka.streams.kstream.KTable; -import org.apache.kafka.streams.kstream.KeyValueMapper; import org.apache.kafka.streams.kstream.Produced; import org.apache.kafka.streams.kstream.Serialized; import org.apache.kafka.streams.kstream.TimeWindows; -import org.apache.kafka.streams.kstream.ValueJoiner; -import org.apache.kafka.streams.kstream.Windowed; import java.util.HashMap; import java.util.Map; import java.util.Properties; +import java.util.concurrent.CountDownLatch; import java.util.concurrent.TimeUnit; /** @@ -83,7 +81,7 @@ static public class RegionCount { public String region; } - public static void main(String[] args) throws Exception { + public static void main(String[] args) { Properties props = new Properties(); props.put(StreamsConfig.APPLICATION_ID_CONFIG, "streams-pageview-typed"); props.put(StreamsConfig.BOOTSTRAP_SERVERS_CONFIG, "localhost:9092"); @@ -151,56 +149,56 @@ public static void main(String[] args) throws Exception { Consumed.with(Serdes.String(), userProfileSerde)); KStream regionCount = views - .leftJoin(users, new ValueJoiner() { - @Override - public PageViewByRegion apply(PageView view, UserProfile profile) { - PageViewByRegion viewByRegion = new PageViewByRegion(); - viewByRegion.user = view.user; - viewByRegion.page = view.page; - - if (profile != null) { - viewByRegion.region = profile.region; - } else { - viewByRegion.region = "UNKNOWN"; - } - return viewByRegion; - } - }) - .map(new KeyValueMapper>() { - @Override - public KeyValue apply(String user, PageViewByRegion viewRegion) { - return new KeyValue<>(viewRegion.region, viewRegion); + .leftJoin(users, (view, profile) -> { + PageViewByRegion viewByRegion = new PageViewByRegion(); + viewByRegion.user = view.user; + viewByRegion.page = view.page; + + if (profile != null) { + viewByRegion.region = profile.region; + } else { + viewByRegion.region = "UNKNOWN"; } + return viewByRegion; }) + .map((user, viewRegion) -> new KeyValue<>(viewRegion.region, viewRegion)) .groupByKey(Serialized.with(Serdes.String(), pageViewByRegionSerde)) .windowedBy(TimeWindows.of(TimeUnit.DAYS.toMillis(7)).advanceBy(TimeUnit.SECONDS.toMillis(1))) .count() .toStream() - .map(new KeyValueMapper, Long, KeyValue>() { - @Override - public KeyValue apply(Windowed key, Long value) { - WindowedPageViewByRegion wViewByRegion = new WindowedPageViewByRegion(); - wViewByRegion.windowStart = key.window().start(); - wViewByRegion.region = key.key(); - - RegionCount rCount = new RegionCount(); - rCount.region = key.key(); - rCount.count = value; - - return new KeyValue<>(wViewByRegion, rCount); - } + .map((key, value) -> { + WindowedPageViewByRegion wViewByRegion = new WindowedPageViewByRegion(); + wViewByRegion.windowStart = key.window().start(); + wViewByRegion.region = key.key(); + + RegionCount rCount = new RegionCount(); + rCount.region = key.key(); + rCount.count = value; + + return new KeyValue<>(wViewByRegion, rCount); }); // write to the result topic regionCount.to("streams-pageviewstats-typed-output", Produced.with(wPageViewByRegionSerde, regionCountSerde)); KafkaStreams streams = new KafkaStreams(builder.build(), props); - streams.start(); - - // usually the stream application would be running forever, - // in this example we just let it run for some time and stop since the input data is finite. - Thread.sleep(5000L); - - streams.close(); + final CountDownLatch latch = new CountDownLatch(1); + + // attach shutdown handler to catch control-c + Runtime.getRuntime().addShutdownHook(new Thread("streams-pipe-shutdown-hook") { + @Override + public void run() { + streams.close(); + latch.countDown(); + } + }); + + try { + streams.start(); + latch.await(); + } catch (Throwable e) { + System.exit(1); + } + System.exit(0); } } diff --git a/streams/examples/src/main/java/org/apache/kafka/streams/examples/pageview/PageViewUntypedDemo.java b/streams/examples/src/main/java/org/apache/kafka/streams/examples/pageview/PageViewUntypedDemo.java index c38d68509a378..dddb542cbaff5 100644 --- a/streams/examples/src/main/java/org/apache/kafka/streams/examples/pageview/PageViewUntypedDemo.java +++ b/streams/examples/src/main/java/org/apache/kafka/streams/examples/pageview/PageViewUntypedDemo.java @@ -33,13 +33,9 @@ import org.apache.kafka.streams.StreamsConfig; import org.apache.kafka.streams.kstream.KStream; import org.apache.kafka.streams.kstream.KTable; -import org.apache.kafka.streams.kstream.KeyValueMapper; import org.apache.kafka.streams.kstream.Produced; import org.apache.kafka.streams.kstream.Serialized; import org.apache.kafka.streams.kstream.TimeWindows; -import org.apache.kafka.streams.kstream.ValueJoiner; -import org.apache.kafka.streams.kstream.ValueMapper; -import org.apache.kafka.streams.kstream.Windowed; import java.util.Properties; @@ -79,46 +75,30 @@ public static void main(String[] args) throws Exception { KTable users = builder.table("streams-userprofile-input", consumed); - KTable userRegions = users.mapValues(new ValueMapper() { - @Override - public String apply(JsonNode record) { - return record.get("region").textValue(); - } - }); + KTable userRegions = users.mapValues(record -> record.get("region").textValue()); KStream regionCount = views - .leftJoin(userRegions, new ValueJoiner() { - @Override - public JsonNode apply(JsonNode view, String region) { - ObjectNode jNode = JsonNodeFactory.instance.objectNode(); - - return jNode.put("user", view.get("user").textValue()) - .put("page", view.get("page").textValue()) - .put("region", region == null ? "UNKNOWN" : region); - } - }) - .map(new KeyValueMapper>() { - @Override - public KeyValue apply(String user, JsonNode viewRegion) { - return new KeyValue<>(viewRegion.get("region").textValue(), viewRegion); - } + .leftJoin(userRegions, (view, region) -> { + ObjectNode jNode = JsonNodeFactory.instance.objectNode(); + return (JsonNode) jNode.put("user", view.get("user").textValue()) + .put("page", view.get("page").textValue()) + .put("region", region == null ? "UNKNOWN" : region); + }) + .map((user, viewRegion) -> new KeyValue<>(viewRegion.get("region").textValue(), viewRegion)) .groupByKey(Serialized.with(Serdes.String(), jsonSerde)) .windowedBy(TimeWindows.of(7 * 24 * 60 * 60 * 1000L).advanceBy(1000)) .count() .toStream() - .map(new KeyValueMapper, Long, KeyValue>() { - @Override - public KeyValue apply(Windowed key, Long value) { - ObjectNode keyNode = JsonNodeFactory.instance.objectNode(); - keyNode.put("window-start", key.window().start()) - .put("region", key.key()); - - ObjectNode valueNode = JsonNodeFactory.instance.objectNode(); - valueNode.put("count", value); - - return new KeyValue<>((JsonNode) keyNode, (JsonNode) valueNode); - } + .map((key, value) -> { + ObjectNode keyNode = JsonNodeFactory.instance.objectNode(); + keyNode.put("window-start", key.window().start()) + .put("region", key.key()); + + ObjectNode valueNode = JsonNodeFactory.instance.objectNode(); + valueNode.put("count", value); + + return new KeyValue<>((JsonNode) keyNode, (JsonNode) valueNode); }); // write to the result topic diff --git a/streams/examples/src/main/java/org/apache/kafka/streams/examples/pipe/PipeDemo.java b/streams/examples/src/main/java/org/apache/kafka/streams/examples/pipe/PipeDemo.java index 538987747c9e2..d61e174a4f42f 100644 --- a/streams/examples/src/main/java/org/apache/kafka/streams/examples/pipe/PipeDemo.java +++ b/streams/examples/src/main/java/org/apache/kafka/streams/examples/pipe/PipeDemo.java @@ -38,7 +38,7 @@ */ public class PipeDemo { - public static void main(String[] args) throws Exception { + public static void main(String[] args) { Properties props = new Properties(); props.put(StreamsConfig.APPLICATION_ID_CONFIG, "streams-pipe"); props.put(StreamsConfig.BOOTSTRAP_SERVERS_CONFIG, "localhost:9092"); diff --git a/streams/examples/src/main/java/org/apache/kafka/streams/examples/temperature/TemperatureDemo.java b/streams/examples/src/main/java/org/apache/kafka/streams/examples/temperature/TemperatureDemo.java index c5eb5f9ec7d89..4607d75d1387b 100644 --- a/streams/examples/src/main/java/org/apache/kafka/streams/examples/temperature/TemperatureDemo.java +++ b/streams/examples/src/main/java/org/apache/kafka/streams/examples/temperature/TemperatureDemo.java @@ -23,10 +23,7 @@ import org.apache.kafka.streams.StreamsBuilder; import org.apache.kafka.streams.StreamsConfig; import org.apache.kafka.streams.kstream.KStream; -import org.apache.kafka.streams.kstream.KeyValueMapper; -import org.apache.kafka.streams.kstream.Predicate; import org.apache.kafka.streams.kstream.Produced; -import org.apache.kafka.streams.kstream.Reducer; import org.apache.kafka.streams.kstream.TimeWindows; import org.apache.kafka.streams.kstream.Windowed; import org.apache.kafka.streams.kstream.WindowedSerdes; @@ -71,7 +68,7 @@ public class TemperatureDemo { // window size within which the filtering is applied private static final int TEMPERATURE_WINDOW_SIZE = 5; - public static void main(String[] args) throws Exception { + public static void main(String[] args) { Properties props = new Properties(); props.put(StreamsConfig.APPLICATION_ID_CONFIG, "streams-temperature"); @@ -89,30 +86,17 @@ public static void main(String[] args) throws Exception { KStream, String> max = source // temperature values are sent without a key (null), so in order // to group and reduce them, a key is needed ("temp" has been chosen) - .selectKey(new KeyValueMapper() { - @Override - public String apply(String key, String value) { - return "temp"; - } - }) + .selectKey((key, value) -> "temp") .groupByKey() .windowedBy(TimeWindows.of(TimeUnit.SECONDS.toMillis(TEMPERATURE_WINDOW_SIZE))) - .reduce(new Reducer() { - @Override - public String apply(String value1, String value2) { - if (Integer.parseInt(value1) > Integer.parseInt(value2)) - return value1; - else - return value2; - } + .reduce((value1, value2) -> { + if (Integer.parseInt(value1) > Integer.parseInt(value2)) + return value1; + else + return value2; }) .toStream() - .filter(new Predicate, String>() { - @Override - public boolean test(Windowed key, String value) { - return Integer.parseInt(value) > TEMPERATURE_THRESHOLD; - } - }); + .filter((key, value) -> Integer.parseInt(value) > TEMPERATURE_THRESHOLD); Serde> windowedSerde = WindowedSerdes.timeWindowedSerdeFrom(String.class); diff --git a/streams/examples/src/main/java/org/apache/kafka/streams/examples/wordcount/WordCountDemo.java b/streams/examples/src/main/java/org/apache/kafka/streams/examples/wordcount/WordCountDemo.java index 7535315d04f9c..4f0150e00582a 100644 --- a/streams/examples/src/main/java/org/apache/kafka/streams/examples/wordcount/WordCountDemo.java +++ b/streams/examples/src/main/java/org/apache/kafka/streams/examples/wordcount/WordCountDemo.java @@ -23,9 +23,7 @@ import org.apache.kafka.streams.StreamsConfig; import org.apache.kafka.streams.kstream.KStream; import org.apache.kafka.streams.kstream.KTable; -import org.apache.kafka.streams.kstream.KeyValueMapper; import org.apache.kafka.streams.kstream.Produced; -import org.apache.kafka.streams.kstream.ValueMapper; import java.util.Arrays; import java.util.Locale; @@ -46,7 +44,7 @@ */ public class WordCountDemo { - public static void main(String[] args) throws Exception { + public static void main(String[] args) { Properties props = new Properties(); props.put(StreamsConfig.APPLICATION_ID_CONFIG, "streams-wordcount"); props.put(StreamsConfig.BOOTSTRAP_SERVERS_CONFIG, "localhost:9092"); @@ -64,18 +62,8 @@ public static void main(String[] args) throws Exception { KStream source = builder.stream("streams-plaintext-input"); KTable counts = source - .flatMapValues(new ValueMapper>() { - @Override - public Iterable apply(String value) { - return Arrays.asList(value.toLowerCase(Locale.getDefault()).split(" ")); - } - }) - .groupBy(new KeyValueMapper() { - @Override - public String apply(String key, String value) { - return value; - } - }) + .flatMapValues(value -> Arrays.asList(value.toLowerCase(Locale.getDefault()).split(" "))) + .groupBy((key, value) -> value) .count(); // need to override value serde to Long type diff --git a/streams/examples/src/main/java/org/apache/kafka/streams/examples/wordcount/WordCountProcessorDemo.java b/streams/examples/src/main/java/org/apache/kafka/streams/examples/wordcount/WordCountProcessorDemo.java index 523bb466e4473..86feaeb1ede66 100644 --- a/streams/examples/src/main/java/org/apache/kafka/streams/examples/wordcount/WordCountProcessorDemo.java +++ b/streams/examples/src/main/java/org/apache/kafka/streams/examples/wordcount/WordCountProcessorDemo.java @@ -26,7 +26,6 @@ import org.apache.kafka.streams.processor.ProcessorContext; import org.apache.kafka.streams.processor.ProcessorSupplier; import org.apache.kafka.streams.processor.PunctuationType; -import org.apache.kafka.streams.processor.Punctuator; import org.apache.kafka.streams.state.KeyValueIterator; import org.apache.kafka.streams.state.KeyValueStore; import org.apache.kafka.streams.state.Stores; @@ -61,19 +60,16 @@ public Processor get() { @SuppressWarnings("unchecked") public void init(final ProcessorContext context) { this.context = context; - this.context.schedule(1000, PunctuationType.STREAM_TIME, new Punctuator() { - @Override - public void punctuate(long timestamp) { - try (KeyValueIterator iter = kvStore.all()) { - System.out.println("----------- " + timestamp + " ----------- "); + this.context.schedule(1000, PunctuationType.STREAM_TIME, timestamp -> { + try (KeyValueIterator iter = kvStore.all()) { + System.out.println("----------- " + timestamp + " ----------- "); - while (iter.hasNext()) { - KeyValue entry = iter.next(); + while (iter.hasNext()) { + KeyValue entry = iter.next(); - System.out.println("[" + entry.key + ", " + entry.value + "]"); + System.out.println("[" + entry.key + ", " + entry.value + "]"); - context.forward(entry.key, entry.value.toString()); - } + context.forward(entry.key, entry.value.toString()); } } }); @@ -103,7 +99,7 @@ public void close() {} } } - public static void main(String[] args) throws Exception { + public static void main(String[] args) { Properties props = new Properties(); props.put(StreamsConfig.APPLICATION_ID_CONFIG, "streams-wordcount-processor"); props.put(StreamsConfig.BOOTSTRAP_SERVERS_CONFIG, "localhost:9092"); @@ -123,7 +119,7 @@ public static void main(String[] args) throws Exception { Stores.inMemoryKeyValueStore("Counts"), Serdes.String(), Serdes.Integer()), - "Process"); + "Process"); builder.addSink("Sink", "streams-wordcount-processor-output", "Process"); diff --git a/streams/quickstart/java/pom.xml b/streams/quickstart/java/pom.xml index fed2bbc46a70e..8381c58c6adc1 100644 --- a/streams/quickstart/java/pom.xml +++ b/streams/quickstart/java/pom.xml @@ -26,11 +26,11 @@ org.apache.kafka streams-quickstart - 1.2.0-SNAPSHOT + 2.0.2-SNAPSHOT .. streams-quickstart-java maven-archetype - \ No newline at end of file + diff --git a/streams/quickstart/java/src/main/resources/archetype-resources/pom.xml b/streams/quickstart/java/src/main/resources/archetype-resources/pom.xml index 6da81a765942b..7158387e343d2 100644 --- a/streams/quickstart/java/src/main/resources/archetype-resources/pom.xml +++ b/streams/quickstart/java/src/main/resources/archetype-resources/pom.xml @@ -29,7 +29,7 @@ UTF-8 - 1.2.0-SNAPSHOT + 2.0.2-SNAPSHOT 1.7.7 1.2.17 @@ -133,4 +133,4 @@ ${kafka.version} - \ No newline at end of file + diff --git a/streams/quickstart/java/src/main/resources/archetype-resources/src/main/java/LineSplit.java b/streams/quickstart/java/src/main/resources/archetype-resources/src/main/java/LineSplit.java index ec40d2aad7548..d712a8392b764 100644 --- a/streams/quickstart/java/src/main/resources/archetype-resources/src/main/java/LineSplit.java +++ b/streams/quickstart/java/src/main/resources/archetype-resources/src/main/java/LineSplit.java @@ -45,22 +45,8 @@ public static void main(String[] args) throws Exception { final StreamsBuilder builder = new StreamsBuilder(); builder.stream("streams-plaintext-input") - .flatMapValues(new ValueMapper>() { - @Override - public Iterable apply(String value) { - return Arrays.asList(value.split("\\W+")); - } - }) - .to("streams-linesplit-output"); - - /* ------- use the code below for Java 8 and uncomment the above ---- - - builder.stream("streams-plaintext-input") - .flatMapValues(value -> Arrays.asList(value.split("\\W+"))) - .to("streams-linesplit-output"); - - ----------------------------------------------------------------- */ - + .flatMapValues(value -> Arrays.asList(value.split("\\W+"))) + .to("streams-linesplit-output"); final Topology topology = builder.build(); final KafkaStreams streams = new KafkaStreams(topology, props); diff --git a/streams/quickstart/java/src/main/resources/archetype-resources/src/main/java/WordCount.java b/streams/quickstart/java/src/main/resources/archetype-resources/src/main/java/WordCount.java index 020eb03ca2b41..bdbefed3be314 100644 --- a/streams/quickstart/java/src/main/resources/archetype-resources/src/main/java/WordCount.java +++ b/streams/quickstart/java/src/main/resources/archetype-resources/src/main/java/WordCount.java @@ -50,26 +50,6 @@ public static void main(String[] args) throws Exception { final StreamsBuilder builder = new StreamsBuilder(); - builder.stream("streams-plaintext-input") - .flatMapValues(new ValueMapper>() { - @Override - public Iterable apply(String value) { - return Arrays.asList(value.toLowerCase(Locale.getDefault()).split("\\W+")); - } - }) - .groupBy(new KeyValueMapper() { - @Override - public String apply(String key, String value) { - return value; - } - }) - .count(Materialized.>as("counts-store")) - .toStream() - .to("streams-wordcount-output", Produced.with(Serdes.String(), Serdes.Long())); - - - /* ------- use the code below for Java 8 and comment the above ---- - builder.stream("streams-plaintext-input") .flatMapValues(value -> Arrays.asList(value.toLowerCase(Locale.getDefault()).split("\\W+"))) .groupBy((key, value) -> value) @@ -77,8 +57,6 @@ public String apply(String key, String value) { .toStream() .to("streams-wordcount-output", Produced.with(Serdes.String(), Serdes.Long())); - ----------------------------------------------------------------- */ - final Topology topology = builder.build(); final KafkaStreams streams = new KafkaStreams(topology, props); final CountDownLatch latch = new CountDownLatch(1); diff --git a/streams/quickstart/pom.xml b/streams/quickstart/pom.xml index e7d1873edb5d5..2e3ecfa6a9645 100644 --- a/streams/quickstart/pom.xml +++ b/streams/quickstart/pom.xml @@ -22,7 +22,7 @@ org.apache.kafka streams-quickstart pom - 1.2.0-SNAPSHOT + 2.0.2-SNAPSHOT Kafka Streams :: Quickstart @@ -118,4 +118,4 @@ - \ No newline at end of file + diff --git a/streams/src/main/java/org/apache/kafka/streams/KafkaStreams.java b/streams/src/main/java/org/apache/kafka/streams/KafkaStreams.java index d6002ff016ba6..ff42a6d54386b 100644 --- a/streams/src/main/java/org/apache/kafka/streams/KafkaStreams.java +++ b/streams/src/main/java/org/apache/kafka/streams/KafkaStreams.java @@ -104,14 +104,14 @@ *

                        * A simple example might look like this: *

                        {@code
                        - * Map props = new HashMap<>();
                        + * Properties props = new Properties();
                          * props.put(StreamsConfig.APPLICATION_ID_CONFIG, "my-stream-processing-application");
                          * props.put(StreamsConfig.BOOTSTRAP_SERVERS_CONFIG, "localhost:9092");
                          * props.put(StreamsConfig.DEFAULT_KEY_SERDE_CLASS_CONFIG, Serdes.String().getClass());
                          * props.put(StreamsConfig.DEFAULT_VALUE_SERDE_CLASS_CONFIG, Serdes.String().getClass());
                          *
                          * StreamsBuilder builder = new StreamsBuilder();
                        - * builder.stream("my-input-topic").mapValues(value -> value.length().toString()).to("my-output-topic");
                        + * builder.stream("my-input-topic").mapValues(value -> String.valueOf(value.length())).to("my-output-topic");
                          *
                          * KafkaStreams streams = new KafkaStreams(builder.build(), props);
                          * streams.start();
                        @@ -599,7 +599,7 @@ public KafkaStreams(final Topology topology,
                             @Deprecated
                             public KafkaStreams(final Topology topology,
                                                 final StreamsConfig config) {
                        -        this(topology.internalTopologyBuilder, config, new DefaultKafkaClientSupplier());
                        +        this(topology, config, new DefaultKafkaClientSupplier());
                             }
                         
                             /**
                        @@ -635,6 +635,10 @@ private KafkaStreams(final InternalTopologyBuilder internalTopologyBuilder,
                                 this.config = config;
                                 this.time = time;
                         
                        +        // adjust the topology if optimization is turned on.
                        +        // TODO: to be removed post 2.0
                        +        internalTopologyBuilder.adjust(config);
                        +
                                 // The application ID is a required config and hence should always have value
                                 processId = UUID.randomUUID();
                                 final String userClientId = config.getString(StreamsConfig.CLIENT_ID_CONFIG);
                        @@ -761,6 +765,7 @@ private static HostInfo parseHostInfo(final String endPoint) {
                              * This function is expected to be called only once during the life cycle of the client.
                              * 

                        * Because threads are started in the background, this method does not block. + * However, if you have global stores in your topology, this method blocks until all global stores are restored. * As a consequence, any fatal exception that happens during processing is by default only logged. * If you want to be notified about dying threads, you can * {@link #setUncaughtExceptionHandler(Thread.UncaughtExceptionHandler) register an uncaught exception handler} diff --git a/streams/src/main/java/org/apache/kafka/streams/StreamsBuilder.java b/streams/src/main/java/org/apache/kafka/streams/StreamsBuilder.java index 517104da323d0..ae6d44c449e60 100644 --- a/streams/src/main/java/org/apache/kafka/streams/StreamsBuilder.java +++ b/streams/src/main/java/org/apache/kafka/streams/StreamsBuilder.java @@ -302,11 +302,10 @@ public synchronized KTable table(final String topic, Objects.requireNonNull(materialized, "materialized can't be null"); final MaterializedInternal> materializedInternal = new MaterializedInternal<>(materialized); materializedInternal.generateStoreNameIfNeeded(internalStreamsBuilder, topic + "-"); + final ConsumedInternal consumedInternal = + new ConsumedInternal<>(Consumed.with(materializedInternal.keySerde(), materializedInternal.valueSerde())); - return internalStreamsBuilder.table(topic, - new ConsumedInternal<>(Consumed.with(materializedInternal.keySerde(), - materializedInternal.valueSerde())), - materializedInternal); + return internalStreamsBuilder.table(topic, consumedInternal, materializedInternal); } /** diff --git a/streams/src/main/java/org/apache/kafka/streams/StreamsConfig.java b/streams/src/main/java/org/apache/kafka/streams/StreamsConfig.java index bc549960de45f..430403ac74da5 100644 --- a/streams/src/main/java/org/apache/kafka/streams/StreamsConfig.java +++ b/streams/src/main/java/org/apache/kafka/streams/StreamsConfig.java @@ -16,8 +16,8 @@ */ package org.apache.kafka.streams; -import org.apache.kafka.clients.admin.AdminClient; import org.apache.kafka.clients.CommonClientConfigs; +import org.apache.kafka.clients.admin.AdminClient; import org.apache.kafka.clients.admin.AdminClientConfig; import org.apache.kafka.clients.consumer.ConsumerConfig; import org.apache.kafka.clients.consumer.KafkaConsumer; @@ -32,10 +32,10 @@ import org.apache.kafka.common.metrics.Sensor; import org.apache.kafka.common.serialization.Serde; import org.apache.kafka.common.serialization.Serdes; +import org.apache.kafka.streams.errors.DefaultProductionExceptionHandler; import org.apache.kafka.streams.errors.DeserializationExceptionHandler; import org.apache.kafka.streams.errors.LogAndFailExceptionHandler; import org.apache.kafka.streams.errors.ProductionExceptionHandler; -import org.apache.kafka.streams.errors.DefaultProductionExceptionHandler; import org.apache.kafka.streams.errors.StreamsException; import org.apache.kafka.streams.processor.DefaultPartitionGrouper; import org.apache.kafka.streams.processor.FailOnInvalidTimestamp; @@ -202,6 +202,16 @@ public class StreamsConfig extends AbstractConfig { @SuppressWarnings("WeakerAccess") public static final String ADMIN_CLIENT_PREFIX = "admin."; + /** + * Config value for parameter (@link #TOPOLOGY_OPTIMIZATION "topology.optimization" for disabling topology optimization + */ + public static final String NO_OPTIMIZATION = "none"; + + /** + * Config value for parameter (@link #TOPOLOGY_OPTIMIZATION "topology.optimization" for enabling topology optimization + */ + public static final String OPTIMIZE = "all"; + /** * Config value for parameter {@link #UPGRADE_FROM_CONFIG "upgrade.from"} for upgrading an application from version {@code 0.10.0.x}. */ @@ -434,6 +444,10 @@ public class StreamsConfig extends AbstractConfig { public static final String STATE_DIR_CONFIG = "state.dir"; private static final String STATE_DIR_DOC = "Directory location for state store."; + /** {@code topology.optimization} */ + public static final String TOPOLOGY_OPTIMIZATION = "topology.optimization"; + private static final String TOPOLOGY_OPTIMIZATION_DOC = "A configuration telling Kafka Streams if it should optimize the topology, disabled by default"; + /** {@code upgrade.from} */ @SuppressWarnings("WeakerAccess") public static final String UPGRADE_FROM_CONFIG = "upgrade.from"; @@ -534,6 +548,12 @@ public class StreamsConfig extends AbstractConfig { CommonClientConfigs.DEFAULT_SECURITY_PROTOCOL, Importance.MEDIUM, CommonClientConfigs.SECURITY_PROTOCOL_DOC) + .define(TOPOLOGY_OPTIMIZATION, + Type.STRING, + NO_OPTIMIZATION, + in(NO_OPTIMIZATION, OPTIMIZE), + Importance.MEDIUM, + TOPOLOGY_OPTIMIZATION_DOC) // LOW @@ -851,10 +871,30 @@ private void checkIfUnexpectedUserSpecifiedConsumerConfig(final Map 5) { + throw new ConfigException(ProducerConfig.MAX_IN_FLIGHT_REQUESTS_PER_CONNECTION, maxInFlightRequestsAsInteger, "Can't exceed 5 when exactly-once processing is enabled"); + } + } } + for (final String config: nonConfigurableConfigs) { if (clientProvidedProps.containsKey(config)) { final String eosMessage = PROCESSING_GUARANTEE_CONFIG + " is set to " + EXACTLY_ONCE + ". Hence, "; @@ -944,25 +984,21 @@ public Map getMainConsumerConfigs(final String groupId, // add admin retries configs for creating topics final AdminClientConfig adminClientDefaultConfig = new AdminClientConfig(getClientPropsWithPrefix(ADMIN_CLIENT_PREFIX, AdminClientConfig.configNames())); consumerProps.put(adminClientPrefix(AdminClientConfig.RETRIES_CONFIG), adminClientDefaultConfig.getInt(AdminClientConfig.RETRIES_CONFIG)); + consumerProps.put(adminClientPrefix(AdminClientConfig.RETRY_BACKOFF_MS_CONFIG), adminClientDefaultConfig.getLong(AdminClientConfig.RETRY_BACKOFF_MS_CONFIG)); // verify that producer batch config is no larger than segment size, then add topic configs required for creating topics final Map topicProps = originalsWithPrefix(TOPIC_PREFIX, false); + final Map producerProps = getClientPropsWithPrefix(PRODUCER_PREFIX, ProducerConfig.configNames()); - if (topicProps.containsKey(topicPrefix(TopicConfig.SEGMENT_INDEX_BYTES_CONFIG))) { - final int segmentSize = Integer.parseInt(topicProps.get(topicPrefix(TopicConfig.SEGMENT_INDEX_BYTES_CONFIG)).toString()); - final Map producerProps = getClientPropsWithPrefix(PRODUCER_PREFIX, ProducerConfig.configNames()); - final int batchSize; - if (producerProps.containsKey(ProducerConfig.BATCH_SIZE_CONFIG)) { - batchSize = Integer.parseInt(producerProps.get(ProducerConfig.BATCH_SIZE_CONFIG).toString()); - } else { - final ProducerConfig producerDefaultConfig = new ProducerConfig(new Properties()); - batchSize = producerDefaultConfig.getInt(ProducerConfig.BATCH_SIZE_CONFIG); - } + if (topicProps.containsKey(topicPrefix(TopicConfig.SEGMENT_BYTES_CONFIG)) && + producerProps.containsKey(ProducerConfig.BATCH_SIZE_CONFIG)) { + final int segmentSize = Integer.parseInt(topicProps.get(topicPrefix(TopicConfig.SEGMENT_BYTES_CONFIG)).toString()); + final int batchSize = Integer.parseInt(producerProps.get(ProducerConfig.BATCH_SIZE_CONFIG).toString()); if (segmentSize < batchSize) { throw new IllegalArgumentException(String.format("Specified topic segment size %d is is smaller than the configured producer batch size %d, this will cause produced batch not able to be appended to the topic", - segmentSize, - batchSize)); + segmentSize, + batchSize)); } } diff --git a/streams/src/main/java/org/apache/kafka/streams/Topology.java b/streams/src/main/java/org/apache/kafka/streams/Topology.java index 22f6ea8362ba0..8b2a46b15c009 100644 --- a/streams/src/main/java/org/apache/kafka/streams/Topology.java +++ b/streams/src/main/java/org/apache/kafka/streams/Topology.java @@ -289,7 +289,7 @@ public synchronized Topology addSource(final String name, * Add a new source that consumes from topics matching the given pattern and forwards the records to child processor * and/or sink nodes. * The source will use the specified key and value deserializers. - * The provided de-/serializers will be used for all matched topics, so care should be taken to specify patterns for + * The provided de-/serializers will be used for all the specified topics, so care should be taken when specifying * topics that share the same key-value data format. * * @param offsetReset the auto offset reset policy to use for this stream if no committed offsets found; @@ -412,8 +412,7 @@ public synchronized Topology addSource(final AutoOffsetReset offsetReset, * @param parentNames the name of one or more source or processor nodes whose output records this sink should consume * and write to its topic * @return itself - * @throws TopologyException if parent processor is not added yet, or if this processor's name is equal to the parent's name, - * or if this processor's name is equal to the parent's name + * @throws TopologyException if parent processor is not added yet, or if this processor's name is equal to the parent's name * @see #addSink(String, String, StreamPartitioner, String...) * @see #addSink(String, String, Serializer, Serializer, String...) * @see #addSink(String, String, Serializer, Serializer, StreamPartitioner, String...) @@ -445,8 +444,7 @@ public synchronized Topology addSink(final String name, * @param parentNames the name of one or more source or processor nodes whose output records this sink should consume * and write to its topic * @return itself - * @throws TopologyException if parent processor is not added yet, or if this processor's name is equal to the parent's name, - * or if this processor's name is equal to the parent's name + * @throws TopologyException if parent processor is not added yet, or if this processor's name is equal to the parent's name * @see #addSink(String, String, String...) * @see #addSink(String, String, Serializer, Serializer, String...) * @see #addSink(String, String, Serializer, Serializer, StreamPartitioner, String...) @@ -474,8 +472,7 @@ public synchronized Topology addSink(final String name, * @param parentNames the name of one or more source or processor nodes whose output records this sink should consume * and write to its topic * @return itself - * @throws TopologyException if parent processor is not added yet, or if this processor's name is equal to the parent's name, - * or if this processor's name is equal to the parent's name + * @throws TopologyException if parent processor is not added yet, or if this processor's name is equal to the parent's name * @see #addSink(String, String, String...) * @see #addSink(String, String, StreamPartitioner, String...) * @see #addSink(String, String, Serializer, Serializer, StreamPartitioner, String...) @@ -505,8 +502,7 @@ public synchronized Topology addSink(final String name, * @param parentNames the name of one or more source or processor nodes whose output records this sink should consume * and write to its topic * @return itself - * @throws TopologyException if parent processor is not added yet, or if this processor's name is equal to the parent's name, - * or if this processor's name is equal to the parent's name + * @throws TopologyException if parent processor is not added yet, or if this processor's name is equal to the parent's name * @see #addSink(String, String, String...) * @see #addSink(String, String, StreamPartitioner, String...) * @see #addSink(String, String, Serializer, Serializer, String...) @@ -533,8 +529,7 @@ public synchronized Topology addSink(final String name, * @param parentNames the name of one or more source or processor nodes whose output records this sink should consume * and dynamically write to topics * @return itself - * @throws TopologyException if parent processor is not added yet, or if this processor's name is equal to the parent's name, - * or if this processor's name is equal to the parent's name + * @throws TopologyException if parent processor is not added yet, or if this processor's name is equal to the parent's name * @see #addSink(String, String, StreamPartitioner, String...) * @see #addSink(String, String, Serializer, Serializer, String...) * @see #addSink(String, String, Serializer, Serializer, StreamPartitioner, String...) @@ -567,8 +562,7 @@ public synchronized Topology addSink(final String name, * @param parentNames the name of one or more source or processor nodes whose output records this sink should consume * and dynamically write to topics * @return itself - * @throws TopologyException if parent processor is not added yet, or if this processor's name is equal to the parent's name, - * or if this processor's name is equal to the parent's name + * @throws TopologyException if parent processor is not added yet, or if this processor's name is equal to the parent's name * @see #addSink(String, String, String...) * @see #addSink(String, String, Serializer, Serializer, String...) * @see #addSink(String, String, Serializer, Serializer, StreamPartitioner, String...) @@ -597,8 +591,7 @@ public synchronized Topology addSink(final String name, * @param parentNames the name of one or more source or processor nodes whose output records this sink should consume * and dynamically write to topics * @return itself - * @throws TopologyException if parent processor is not added yet, or if this processor's name is equal to the parent's name, - * or if this processor's name is equal to the parent's name + * @throws TopologyException if parent processor is not added yet, or if this processor's name is equal to the parent's name * @see #addSink(String, String, String...) * @see #addSink(String, String, StreamPartitioner, String...) * @see #addSink(String, String, Serializer, Serializer, StreamPartitioner, String...) @@ -629,8 +622,7 @@ public synchronized Topology addSink(final String name, * @param parentNames the name of one or more source or processor nodes whose output records this sink should consume * and dynamically write to topics * @return itself - * @throws TopologyException if parent processor is not added yet, or if this processor's name is equal to the parent's name, - * or if this processor's name is equal to the parent's name + * @throws TopologyException if parent processor is not added yet, or if this processor's name is equal to the parent's name * @see #addSink(String, String, String...) * @see #addSink(String, String, StreamPartitioner, String...) * @see #addSink(String, String, Serializer, Serializer, String...) @@ -655,8 +647,7 @@ public synchronized Topology addSink(final String name, * @param parentNames the name of one or more source or processor nodes whose output records this processor should receive * and process * @return itself - * @throws TopologyException if parent processor is not added yet, or if this processor's name is equal to the parent's name, - * or if this processor's name is equal to the parent's name + * @throws TopologyException if parent processor is not added yet, or if this processor's name is equal to the parent's name */ public synchronized Topology addProcessor(final String name, final ProcessorSupplier supplier, @@ -776,5 +767,4 @@ public synchronized Topology connectProcessorAndStateStores(final String process public synchronized TopologyDescription describe() { return internalTopologyBuilder.describe(); } - } diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/Consumed.java b/streams/src/main/java/org/apache/kafka/streams/kstream/Consumed.java index e2132ec79a0c4..0af7dbea7b4ac 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/Consumed.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/Consumed.java @@ -97,7 +97,7 @@ public static Consumed with(final Serde keySerde, /** * Create an instance of {@link Consumed} with key and value {@link Serde}s. * - * @param keySerde the key serde. If {@code null}the default key serde from config will be used + * @param keySerde the key serde. If {@code null} the default key serde from config will be used * @param valueSerde the value serde. If {@code null} the default value serde from config will be used * @param key type * @param value type diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/KGroupedStream.java b/streams/src/main/java/org/apache/kafka/streams/kstream/KGroupedStream.java index 53a2be79add13..7b69e0317ea34 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/KGroupedStream.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/KGroupedStream.java @@ -126,7 +126,7 @@ public interface KGroupedStream { * aggregate and the record's value. * If there is no current aggregate the {@link Reducer} is not applied and the new aggregate will be the record's * value as-is. - * Thus, {@code reduce(Reducer, String)} can be used to compute aggregate functions like sum, min, or max. + * Thus, {@code reduce(Reducer)} can be used to compute aggregate functions like sum, min, or max. *

                        * Not all updates might get sent downstream, as an internal cache is used to deduplicate consecutive updates to * the same key. @@ -189,7 +189,7 @@ public interface KGroupedStream { *

                        {@code
                              * KafkaStreams streams = ... // compute sum
                              * String queryableStoreName = "storeName" // the store name should be the name of the store as defined by the Materialized instance
                        -     * ReadOnlyKeyValueStore localStore = streams.store(queryableStoreName, QueryableStoreTypes.keyValueStore());
                        +     * ReadOnlyKeyValueStore localStore = streams.store(queryableStoreName, QueryableStoreTypes.keyValueStore());
                              * String key = "some-key";
                              * Long sumForKey = localStore.get(key); // key must be local (application state is shared over all running Kafka Streams instances)
                              * }
                        @@ -271,7 +271,7 @@ KTable aggregate(final Initializer initializer, * The specified {@link Aggregator} is applied for each input record and computes a new aggregate using the current * aggregate (or for the very first record using the intermediate aggregation result provided via the * {@link Initializer}) and the record's value. - * Thus, {@code aggregate(Initializer, Aggregator, Serde, String)} can be used to compute aggregate functions like + * Thus, {@code aggregate(Initializer, Aggregator, Materialized)} can be used to compute aggregate functions like * count (c.f. {@link #count()}). *

                        * Not all updates might get sent downstream, as an internal cache is used to deduplicate consecutive updates to @@ -286,7 +286,7 @@ KTable aggregate(final Initializer initializer, *

                        {@code
                              * KafkaStreams streams = ... // some aggregation on value type double
                              * String queryableStoreName = "storeName" // the store name should be the name of the store as defined by the Materialized instance
                        -     * ReadOnlyKeyValueStore localStore = streams.store(queryableStoreName, QueryableStoreTypes.keyValueStore());
                        +     * ReadOnlyKeyValueStore localStore = streams.store(queryableStoreName, QueryableStoreTypes.keyValueStore());
                              * String key = "some-key";
                              * Long aggForKey = localStore.get(key); // key must be local (application state is shared over all running Kafka Streams instances)
                              * }
                        diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/KGroupedTable.java b/streams/src/main/java/org/apache/kafka/streams/kstream/KGroupedTable.java index 0e263362f5ced..30f348c2eaf34 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/KGroupedTable.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/KGroupedTable.java @@ -60,7 +60,7 @@ public interface KGroupedTable { * {@link KafkaStreams#store(String, QueryableStoreType) KafkaStreams#store(...)}: *
                        {@code
                              * KafkaStreams streams = ... // counting words
                        -     * ReadOnlyKeyValueStore localStore = streams.store(queryableStoreName, QueryableStoreTypes.keyValueStore());
                        +     * ReadOnlyKeyValueStore localStore = streams.store(queryableStoreName, QueryableStoreTypes.keyValueStore());
                              * String key = "some-word";
                              * Long countForWord = localStore.get(key); // key must be local (application state is shared over all running Kafka Streams instances)
                              * }
                        @@ -89,7 +89,6 @@ public interface KGroupedTable { * the same key into a new instance of {@link KTable}. * Records with {@code null} key are ignored. * The result is written into a local {@link KeyValueStore} (which is basically an ever-updating materialized view) - * that can be queried using the provided {@code queryableStoreName}. * Furthermore, updates to the store are sent downstream into a {@link KTable} changelog stream. *

                        * Not all updates might get sent downstream, as an internal cache is used to deduplicate consecutive updates to @@ -158,7 +157,7 @@ public interface KGroupedTable { * {@link KafkaStreams#store(String, QueryableStoreType) KafkaStreams#store(...)}: *

                        {@code
                              * KafkaStreams streams = ... // counting words
                        -     * ReadOnlyKeyValueStore localStore = streams.store(queryableStoreName, QueryableStoreTypes.keyValueStore());
                        +     * ReadOnlyKeyValueStore localStore = streams.store(queryableStoreName, QueryableStoreTypes.keyValueStore());
                              * String key = "some-word";
                              * Long countForWord = localStore.get(key); // key must be local (application state is shared over all running Kafka Streams instances)
                              * }
                        @@ -191,7 +190,6 @@ KTable reduce(final Reducer adder, * Combining implies that the type of the aggregate result is the same as the type of the input value * (c.f. {@link #aggregate(Initializer, Aggregator, Aggregator)}). * The result is written into a local {@link KeyValueStore} (which is basically an ever-updating materialized view) - * that can be queried using the provided {@code queryableStoreName}. * Furthermore, updates to the store are sent downstream into a {@link KTable} changelog stream. *

                        * Each update to the original {@link KTable} results in a two step update of the result {@link KTable}. @@ -202,7 +200,7 @@ KTable reduce(final Reducer adder, * record from the aggregate. * If there is no current aggregate the {@link Reducer} is not applied and the new aggregate will be the record's * value as-is. - * Thus, {@code reduce(Reducer, Reducer, String)} can be used to compute aggregate functions like sum. + * Thus, {@code reduce(Reducer, Reducer)} can be used to compute aggregate functions like sum. * For sum, the adder and subtractor would work as follows: *

                        {@code
                              * public class SumAdder implements Reducer {
                        @@ -243,12 +241,12 @@ KTable reduce(final Reducer adder,
                         
                             /**
                              * Aggregate the value of records of the original {@link KTable} that got {@link KTable#groupBy(KeyValueMapper)
                        -     * mapped} to the same key into a new instance of {@link KTable} using default serializers and deserializers.
                        +     * mapped} to the same key into a new instance of {@link KTable}.
                              * Records with {@code null} key are ignored.
                              * Aggregating is a generalization of {@link #reduce(Reducer, Reducer, Materialized) combining via reduce(...)} as it,
                              * for example, allows the result to have a different type than the input values.
                              * The result is written into a local {@link KeyValueStore} (which is basically an ever-updating materialized view)
                        -     * provided by the given {@code storeSupplier}.
                        +     * that can be queried using the provided {@code queryableStoreName}.
                              * Furthermore, updates to the store are sent downstream into a {@link KTable} changelog stream.
                              * 

                        * The specified {@link Initializer} is applied once directly before the first input record is processed to @@ -260,11 +258,11 @@ KTable reduce(final Reducer adder, * The specified {@link Aggregator subtractor} is applied for each "replaced" record of the original {@link KTable} * and computes a new aggregate using the current aggregate and the record's value by "removing" the "replaced" * record from the aggregate. - * Thus, {@code aggregate(Initializer, Aggregator, Aggregator, String)} can be used to compute aggregate functions + * Thus, {@code aggregate(Initializer, Aggregator, Aggregator, Materialized)} can be used to compute aggregate functions * like sum. * For sum, the initializer, adder, and subtractor would work as follows: *

                        {@code
                        -     * // in this example, LongSerde.class must be set as default value serde in StreamsConfig
                        +     * // in this example, LongSerde.class must be set as value serde in Materialized#withValueSerde
                              * public class SumInitializer implements Initializer {
                              *   public Long apply() {
                              *     return 0L;
                        @@ -277,7 +275,7 @@ KTable reduce(final Reducer adder,
                              *   }
                              * }
                              *
                        -     * public class SumSubstractor implements Aggregator {
                        +     * public class SumSubtractor implements Aggregator {
                              *   public Long apply(String key, Integer oldValue, Long aggregate) {
                              *     return aggregate - oldValue;
                              *   }
                        @@ -294,7 +292,7 @@ KTable reduce(final Reducer adder,
                              * {@link KafkaStreams#store(String, QueryableStoreType) KafkaStreams#store(...)}:
                              * 
                        {@code
                              * KafkaStreams streams = ... // counting words
                        -     * ReadOnlyKeyValueStore localStore = streams.store(queryableStoreName, QueryableStoreTypes.keyValueStore());
                        +     * ReadOnlyKeyValueStore localStore = streams.store(queryableStoreName, QueryableStoreTypes.keyValueStore());
                              * String key = "some-word";
                              * Long countForWord = localStore.get(key); // key must be local (application state is shared over all running Kafka Streams instances)
                              * }
                        @@ -333,7 +331,6 @@ KTable aggregate(final Initializer initializer, * If the result value type does not match the {@link StreamsConfig#DEFAULT_VALUE_SERDE_CLASS_CONFIG default value * serde} you should use {@link #aggregate(Initializer, Aggregator, Aggregator, Materialized)}. * The result is written into a local {@link KeyValueStore} (which is basically an ever-updating materialized view) - * provided by the given {@code storeSupplier}. * Furthermore, updates to the store are sent downstream into a {@link KTable} changelog stream. *

                        * The specified {@link Initializer} is applied once directly before the first input record is processed to @@ -362,7 +359,7 @@ KTable aggregate(final Initializer initializer, * } * } * - * public class SumSubstractor implements Aggregator { + * public class SumSubtractor implements Aggregator { * public Long apply(String key, Integer oldValue, Long aggregate) { * return aggregate - oldValue; * } 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 index da86a75b9e3fb..ae3b28a35ce0e 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/KStream.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/KStream.java @@ -190,7 +190,7 @@ public interface KStream { * The provided {@link ValueMapperWithKey} is applied to each input record value and computes a new value for it. * Thus, an input record {@code } can be transformed into an output record {@code }. * This is a stateless record-by-record operation (cf. - * {@link #transformValues(ValueTransformerSupplier, String...)} for stateful value transformation). + * {@link #transformValues(ValueTransformerWithKeySupplier, String...)} for stateful value transformation). *

                        * The example below counts the number of tokens of key and value strings. *

                        {@code
                        @@ -317,7 +317,7 @@ public interface KStream {
                              * stream (value type can be altered arbitrarily).
                              * The provided {@link ValueMapperWithKey} is applied to each input record and computes zero or more output values.
                              * Thus, an input record {@code } can be transformed into output records {@code , , ...}.
                        -     * This is a stateless record-by-record operation (cf. {@link #transformValues(ValueTransformerSupplier, String...)}
                        +     * This is a stateless record-by-record operation (cf. {@link #transformValues(ValueTransformerWithKeySupplier, String...)}
                              * for stateful value transformation).
                              * 

                        * The example below splits input records {@code }, with key=1, containing sentences as values @@ -417,8 +417,8 @@ public interface KStream { KStream merge(final KStream stream); /** - * Materialize this stream to a topic and creates a new {@code KStream} from the topic using default serializers and - * deserializers and producer's {@link DefaultPartitioner}. + * Materialize this stream to a topic and creates a new {@code KStream} from the topic using default serializers, + * deserializers, and producer's {@link DefaultPartitioner}. * The specified topic should be manually created before it is used (i.e., before the Kafka Streams application is * started). *

                        @@ -440,8 +440,8 @@ public interface KStream { * This is equivalent to calling {@link #to(String, Produced) to(someTopic, Produced.with(keySerde, valueSerde)} * and {@link StreamsBuilder#stream(String, Consumed) StreamsBuilder#stream(someTopicName, Consumed.with(keySerde, valueSerde))}. * - * @param topic - * @param produced + * @param topic the topic name + * @param produced the options to use when producing to the topic * @return a {@code KStream} that contains the exact same (and potentially repartitioned) records as this {@code KStream} */ KStream through(final String topic, @@ -947,8 +947,9 @@ KStream join(final KStream otherStream, final JoinWindows windows); /** - * Join records of this stream with another {@code KStream}'s records using windowed inner equi join with default - * serializers and deserializers. + * Join records of this stream with another {@code KStream}'s records using windowed inner equi join using the + * {@link Joined} instance for configuration of the {@link Serde key serde}, {@link Serde this stream's value serde}, + * and {@link Serde the other stream's value serde}. * The join is computed on the records' key with join attribute {@code thisKStream.key == otherKStream.key}. * Furthermore, two records are only joined if their timestamps are close to each other as defined by the given * {@link JoinWindows}, i.e., the window defines an additional join predicate on the record timestamps. @@ -1106,8 +1107,9 @@ KStream leftJoin(final KStream otherStream, final JoinWindows windows); /** - * Join records of this stream with another {@code KStream}'s records using windowed left equi join with default - * serializers and deserializers. + * Join records of this stream with another {@code KStream}'s records using windowed left equi join using the + * {@link Joined} instance for configuration of the {@link Serde key serde}, {@link Serde this stream's value serde}, + * and {@link Serde the other stream's value serde}. * In contrast to {@link #join(KStream, ValueJoiner, JoinWindows) inner-join}, all records from this stream will * produce at least one output record (cf. below). * The join is computed on the records' key with join attribute {@code thisKStream.key == otherKStream.key}. @@ -1270,8 +1272,9 @@ KStream outerJoin(final KStream otherStream, final JoinWindows windows); /** - * Join records of this stream with another {@code KStream}'s records using windowed outer equi join with default - * serializers and deserializers. + * Join records of this stream with another {@code KStream}'s records using windowed outer equi join using the + * {@link Joined} instance for configuration of the {@link Serde key serde}, {@link Serde this stream's value serde}, + * and {@link Serde the other stream's value serde}. * In contrast to {@link #join(KStream, ValueJoiner, JoinWindows) inner-join} or * {@link #leftJoin(KStream, ValueJoiner, JoinWindows) left-join}, all records from both streams will produce at * least one output record (cf. below). diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/KTable.java b/streams/src/main/java/org/apache/kafka/streams/kstream/KTable.java index da540ba627f7e..bdd6dc3b37a21 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/KTable.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/KTable.java @@ -70,7 +70,7 @@ public interface KTable { /** * Create a new {@code KTable} that consists of all records of this {@code KTable} which satisfy the given - * predicate. + * predicate, with default serializers, deserializers, and state store. * All records that do not satisfy the predicate are dropped. * For each {@code KTable} update, the filter is evaluated based on the current update * record and then an update record is produced for the result {@code KTable}. @@ -92,7 +92,8 @@ public interface KTable { /** * Create a new {@code KTable} that consists of all records of this {@code KTable} which satisfy the given - * predicate. + * predicate, with the {@link Serde key serde}, {@link Serde value serde}, and the underlying + * {@link KeyValueStore materialized state storage} configured in the {@link Materialized} instance. * All records that do not satisfy the predicate are dropped. * For each {@code KTable} update, the filter is evaluated based on the current update * record and then an update record is produced for the result {@code KTable}. @@ -130,7 +131,7 @@ KTable filter(final Predicate predicate, /** * Create a new {@code KTable} that consists all records of this {@code KTable} which do not satisfy the - * given predicate. + * given predicate, with default serializers, deserializers, and state store. * All records that do satisfy the predicate are dropped. * For each {@code KTable} update, the filter is evaluated based on the current update * record and then an update record is produced for the result {@code KTable}. @@ -152,7 +153,8 @@ KTable filter(final Predicate predicate, /** * Create a new {@code KTable} that consists all records of this {@code KTable} which do not satisfy the - * given predicate. + * given predicate, with the {@link Serde key serde}, {@link Serde value serde}, and the underlying + * {@link KeyValueStore materialized state storage} configured in the {@link Materialized} instance. * All records that do satisfy the predicate are dropped. * For each {@code KTable} update, the filter is evaluated based on the current update * record and then an update record is produced for the result {@code KTable}. @@ -190,7 +192,7 @@ KTable filterNot(final Predicate predicate, /** * Create a new {@code KTable} by transforming the value of each record in this {@code KTable} into a new value - * (with possible new type) in the new {@code KTable}. + * (with possibly a new type) in the new {@code KTable}, with default serializers, deserializers, and state store. * For each {@code KTable} update the provided {@link ValueMapper} is applied to the value of the updated record and * computes a new value for it, resulting in an updated record for the result {@code KTable}. * Thus, an input record {@code } can be transformed into an output record {@code }. @@ -224,7 +226,7 @@ KTable filterNot(final Predicate predicate, /** * Create a new {@code KTable} by transforming the value of each record in this {@code KTable} into a new value - * (with possible new type) in the new {@code KTable}. + * (with possibly a new type) in the new {@code KTable}, with default serializers, deserializers, and state store. * For each {@code KTable} update the provided {@link ValueMapperWithKey} is applied to the value of the update * record and computes a new value for it, resulting in an updated record for the result {@code KTable}. * Thus, an input record {@code } can be transformed into an output record {@code }. @@ -259,7 +261,9 @@ KTable filterNot(final Predicate predicate, /** * Create a new {@code KTable} by transforming the value of each record in this {@code KTable} into a new value - * (with possible new type) in the new {@code KTable}. + * (with possibly a new type) in the new {@code KTable}, with the {@link Serde key serde}, {@link Serde value serde}, + * and the underlying {@link KeyValueStore materialized state storage} configured in the {@link Materialized} + * instance. * For each {@code KTable} update the provided {@link ValueMapper} is applied to the value of the updated record and * computes a new value for it, resulting in an updated record for the result {@code KTable}. * Thus, an input record {@code } can be transformed into an output record {@code }. @@ -303,7 +307,9 @@ KTable mapValues(final ValueMapper mapper, /** * Create a new {@code KTable} by transforming the value of each record in this {@code KTable} into a new value - * (with possible new type) in the new {@code KTable}. + * (with possibly a new type) in the new {@code KTable}, with the {@link Serde key serde}, {@link Serde value serde}, + * and the underlying {@link KeyValueStore materialized state storage} configured in the {@link Materialized} + * instance. * For each {@code KTable} update the provided {@link ValueMapperWithKey} is applied to the value of the update * record and computes a new value for it, resulting in an updated record for the result {@code KTable}. * Thus, an input record {@code } can be transformed into an output record {@code }. @@ -384,8 +390,8 @@ KTable mapValues(final ValueMapperWithKey KStream toStream(final KeyValueMapper mapper); /** - * Create a new {@code KTable} by transforming the value of each record in this {@code KTable} into a new value, - * (with possibly new type). + * Create a new {@code KTable} by transforming the value of each record in this {@code KTable} into a new value + * (with possibly a new type), with default serializers, deserializers, and state store. * A {@link ValueTransformerWithKey} (provided by the given {@link ValueTransformerWithKeySupplier}) is applied to each input * record value and computes a new value for it. * Thus, an input record {@code } can be transformed into an output record {@code }. @@ -457,8 +463,9 @@ KTable transformValues(final ValueTransformerWithKeySupplier KGroupedTable groupBy(final KeyValueMapper serialized); /** - * Join records of this {@code KTable} with another {@code KTable}'s records using non-windowed inner equi join. + * Join records of this {@code KTable} with another {@code KTable}'s records using non-windowed inner equi join, + * with default serializers, deserializers, and state store. * The join is a primary key join with join attribute {@code thisKTable.key == otherKTable.key}. * The result is an ever updating {@code KTable} that represents the current (i.e., processing time) result * of the join. @@ -672,7 +680,9 @@ KTable join(final KTable other, final ValueJoiner joiner); /** - * Join records of this {@code KTable} with another {@code KTable}'s records using non-windowed inner equi join. + * Join records of this {@code KTable} with another {@code KTable}'s records using non-windowed inner equi join, + * with the {@link Materialized} instance for configuration of the {@link Serde key serde}, + * {@link Serde the result table's value serde}, and {@link KeyValueStore state store}. * The join is a primary key join with join attribute {@code thisKTable.key == otherKTable.key}. * The result is an ever updating {@code KTable} that represents the current (i.e., processing time) result * of the join. @@ -751,7 +761,7 @@ KTable join(final KTable other, /** * Join records of this {@code KTable} (left input) with another {@code KTable}'s (right input) records using - * non-windowed left equi join. + * non-windowed left equi join, with default serializers, deserializers, and state store. * The join is a primary key join with join attribute {@code thisKTable.key == otherKTable.key}. * In contrast to {@link #join(KTable, ValueJoiner) inner-join}, all records from left {@code KTable} will produce * an output record (cf. below). @@ -833,7 +843,8 @@ KTable leftJoin(final KTable other, /** * Join records of this {@code KTable} (left input) with another {@code KTable}'s (right input) records using - * non-windowed left equi join. + * non-windowed left equi join, with the {@link Materialized} instance for configuration of the {@link Serde key serde}, + * {@link Serde the result table's value serde}, and {@link KeyValueStore state store}. * The join is a primary key join with join attribute {@code thisKTable.key == otherKTable.key}. * In contrast to {@link #join(KTable, ValueJoiner) inner-join}, all records from left {@code KTable} will produce * an output record (cf. below). @@ -919,7 +930,7 @@ KTable leftJoin(final KTable other, /** * Join records of this {@code KTable} (left input) with another {@code KTable}'s (right input) records using - * non-windowed outer equi join. + * non-windowed outer equi join, with default serializers, deserializers, and state store. * The join is a primary key join with join attribute {@code thisKTable.key == otherKTable.key}. * In contrast to {@link #join(KTable, ValueJoiner) inner-join} or {@link #leftJoin(KTable, ValueJoiner) left-join}, * all records from both input {@code KTable}s will produce an output record (cf. below). @@ -1000,7 +1011,8 @@ KTable outerJoin(final KTable other, /** * Join records of this {@code KTable} (left input) with another {@code KTable}'s (right input) records using - * non-windowed outer equi join. + * non-windowed outer equi join, with the {@link Materialized} instance for configuration of the {@link Serde key serde}, + * {@link Serde the result table's value serde}, and {@link KeyValueStore state store}. * The join is a primary key join with join attribute {@code thisKTable.key == otherKTable.key}. * In contrast to {@link #join(KTable, ValueJoiner) inner-join} or {@link #leftJoin(KTable, ValueJoiner) left-join}, * all records from both input {@code KTable}s will produce an output record (cf. below). 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 index 43b6115505eaf..0ab34699cf70a 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/Transformer.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/Transformer.java @@ -52,7 +52,7 @@ public interface Transformer { * Initialize this transformer. * This is called once per instance when the topology gets initialized. * When the framework is done with the transformer, {@link #close()} will be called on it; the - * framework may later re-use the transformer by calling {@link #init()} again. + * framework may later re-use the transformer by calling {@link #init(ProcessorContext)} again. *

                        * The provided {@link ProcessorContext context} can be used to access topology and record meta data, to * {@link ProcessorContext#schedule(long, PunctuationType, Punctuator) schedule} a method to be @@ -73,7 +73,7 @@ public interface Transformer { *

                        * If more than one output record should be forwarded downstream {@link ProcessorContext#forward(Object, Object)} * and {@link ProcessorContext#forward(Object, Object, To)} can be used. - * If not record should be forwarded downstream, {@code transform} can return {@code null}. + * If record should not be forwarded downstream, {@code transform} can return {@code null}. * * @param key the key for the record * @param value the value for the record @@ -84,7 +84,7 @@ public interface Transformer { /** * Close this transformer and clean up any resources. The framework may - * later re-use this transformer by calling {@link #init()} on it again. + * later re-use this transformer by calling {@link #init(ProcessorContext)} on it again. *

                        * To generate new {@link KeyValue} pairs {@link ProcessorContext#forward(Object, Object)} and * {@link ProcessorContext#forward(Object, Object, To)} can be used. 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 index 866cce8936182..b02311bf95890 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/ValueTransformer.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/ValueTransformer.java @@ -51,7 +51,7 @@ public interface ValueTransformer { * Initialize this transformer. * This is called once per instance when the topology gets initialized. * When the framework is done with the transformer, {@link #close()} will be called on it; the - * framework may later re-use the transformer by calling {@link #init()} again. + * framework may later re-use the transformer by calling {@link #init(ProcessorContext)} again. *

                        * The provided {@link ProcessorContext context} can be used to access topology and record meta data, to * {@link ProcessorContext#schedule(long, PunctuationType, Punctuator) schedule} a method to be @@ -87,7 +87,7 @@ public interface ValueTransformer { /** * Close this transformer and clean up any resources. The framework may - * later re-use this transformer by calling {@link #init()} on it again. + * later re-use this transformer by calling {@link #init(ProcessorContext)} on it again. *

                        * It is not possible to return any new output records within {@code close()}. * Using {@link ProcessorContext#forward(Object, Object)} or {@link ProcessorContext#forward(Object, Object, To)} 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 index 08540a1aadfaa..f6250683282c8 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/Window.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/Window.java @@ -103,8 +103,8 @@ public int hashCode() { @Override public String toString() { return "Window{" + - "start=" + startMs + - ", end=" + endMs + + "startMs=" + startMs + + ", endMs=" + endMs + '}'; } } diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/WindowedSerdes.java b/streams/src/main/java/org/apache/kafka/streams/kstream/WindowedSerdes.java index d0381c787c05a..6a851a10d9d38 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/WindowedSerdes.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/WindowedSerdes.java @@ -54,6 +54,6 @@ static public Serde> timeWindowedSerdeFrom(final Class type) * Construct a {@code SessionWindowedSerde} object for the specified inner class type. */ static public Serde> sessionWindowedSerdeFrom(final Class type) { - return new TimeWindowedSerde<>(Serdes.serdeFrom(type)); + return new SessionWindowedSerde<>(Serdes.serdeFrom(type)); } } diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/InternalStreamsBuilder.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/InternalStreamsBuilder.java index 0a19b4eb0c0dd..c7bf2fac8f7ec 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/InternalStreamsBuilder.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/InternalStreamsBuilder.java @@ -72,11 +72,7 @@ public KStream stream(final Pattern topicPattern, final ConsumedInt public KTable table(final String topic, final ConsumedInternal consumed, final MaterializedInternal> materialized) { - // explicitly disable logging for source table materialized stores - materialized.withLoggingDisabled(); - - final StoreBuilder> storeBuilder = new KeyValueStoreMaterializer<>(materialized) - .materialize(); + final StoreBuilder> storeBuilder = new KeyValueStoreMaterializer<>(materialized).materialize(); final String source = newProcessorName(KStreamImpl.SOURCE_NAME); final String name = newProcessorName(KTableImpl.SOURCE_NAME); @@ -88,7 +84,7 @@ public KTable table(final String topic, name); internalTopologyBuilder.addStateStore(storeBuilder, name); - internalTopologyBuilder.connectSourceStoreAndTopic(storeBuilder.name(), topic); + internalTopologyBuilder.markSourceStoreAndTopic(storeBuilder, topic); return kTable; } 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 index 7356aff153fb3..d31d7cf1c12b3 100644 --- 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 @@ -564,8 +564,9 @@ private KStream globalTableJoin(final GlobalKTable g final KTableValueGetterSupplier valueGetterSupplier = ((GlobalKTableImpl) globalTable).valueGetterSupplier(); final String name = builder.newProcessorName(LEFTJOIN_NAME); + builder.internalTopologyBuilder.addProcessor(name, new KStreamGlobalKTableJoin<>(valueGetterSupplier, joiner, keyMapper, leftJoin), this.name); - return new KStreamImpl<>(builder, name, sourceNodes, false); + return new KStreamImpl<>(builder, name, sourceNodes, repartitionRequired); } @SuppressWarnings("unchecked") @@ -580,7 +581,6 @@ private KStream doStreamTableJoin(final KTable other, final String name = builder.newProcessorName(leftJoin ? LEFTJOIN_NAME : JOIN_NAME); builder.internalTopologyBuilder.addProcessor(name, new KStreamKTableJoin<>(((KTableImpl) other).valueGetterSupplier(), joiner, leftJoin), this.name); builder.internalTopologyBuilder.connectProcessorAndStateStores(name, ((KTableImpl) other).valueGetterSupplier().storeNames()); - builder.internalTopologyBuilder.connectProcessors(this.name, ((KTableImpl) other).name); return new KStreamImpl<>(builder, name, allSourceNodes, false); } diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamReduce.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamReduce.java index 9f404ea55f68d..8d078d1d083ef 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamReduce.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamReduce.java @@ -59,7 +59,6 @@ private class KStreamReduceProcessor extends AbstractProcessor { public void init(final ProcessorContext context) { super.init(context); metrics = (StreamsMetricsImpl) context.metrics(); - store = (KeyValueStore) context.getStateStore(storeName); tupleForwarder = new TupleForwarder<>(store, context, new ForwardingCacheFlushListener(context, sendOldValues), sendOldValues); } diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableAggregate.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableAggregate.java index b60f9ab1f73e5..b00a09cd25125 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableAggregate.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableAggregate.java @@ -75,22 +75,29 @@ public void process(final K key, final Change value) { throw new StreamsException("Record key for KTable aggregate operator with state " + storeName + " should not be null."); } - T oldAgg = store.get(key); - - if (oldAgg == null) { - oldAgg = initializer.apply(); - } - - T newAgg = oldAgg; + final T oldAgg = store.get(key); + final T intermediateAgg; // first try to remove the old value - if (value.oldValue != null) { - newAgg = remove.apply(key, value.oldValue, newAgg); + if (value.oldValue != null && oldAgg != null) { + intermediateAgg = remove.apply(key, value.oldValue, oldAgg); + } else { + intermediateAgg = oldAgg; } // then try to add the new value + final T newAgg; if (value.newValue != null) { - newAgg = add.apply(key, value.newValue, newAgg); + final T initializedAgg; + if (intermediateAgg == null) { + initializedAgg = initializer.apply(); + } else { + initializedAgg = intermediateAgg; + } + + newAgg = add.apply(key, value.newValue, initializedAgg); + } else { + newAgg = intermediateAgg; } // update the store with the new value diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableReduce.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableReduce.java index 069b360463185..bee89877b04d1 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableReduce.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableReduce.java @@ -71,20 +71,25 @@ public void process(final K key, final Change value) { } final V oldAgg = store.get(key); - V newAgg = oldAgg; + final V intermediateAgg; - // first try to add the new value + // first try to remove the old value + if (value.oldValue != null && oldAgg != null) { + intermediateAgg = removeReducer.apply(oldAgg, value.oldValue); + } else { + intermediateAgg = oldAgg; + } + + // then try to add the new value + final V newAgg; if (value.newValue != null) { - if (newAgg == null) { + if (intermediateAgg == null) { newAgg = value.newValue; } else { - newAgg = addReducer.apply(newAgg, value.newValue); + newAgg = addReducer.apply(intermediateAgg, value.newValue); } - } - - // then try to remove the old value - if (value.oldValue != null) { - newAgg = removeReducer.apply(newAgg, value.oldValue); + } else { + newAgg = intermediateAgg; } // update the store with the new value diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/TaskId.java b/streams/src/main/java/org/apache/kafka/streams/processor/TaskId.java index f4c9ce00814c1..01feb77f5efb9 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/TaskId.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/TaskId.java @@ -43,19 +43,19 @@ public String toString() { } /** - * @throws TaskIdFormatException if the string is not a valid {@link TaskId} + * @throws TaskIdFormatException if the taskIdStr is not a valid {@link TaskId} */ - public static TaskId parse(String string) { - int index = string.indexOf('_'); - if (index <= 0 || index + 1 >= string.length()) throw new TaskIdFormatException(string); + public static TaskId parse(final String taskIdStr) { + final int index = taskIdStr.indexOf('_'); + if (index <= 0 || index + 1 >= taskIdStr.length()) throw new TaskIdFormatException(taskIdStr); try { - int topicGroupId = Integer.parseInt(string.substring(0, index)); - int partition = Integer.parseInt(string.substring(index + 1)); + final int topicGroupId = Integer.parseInt(taskIdStr.substring(0, index)); + final int partition = Integer.parseInt(taskIdStr.substring(index + 1)); return new TaskId(topicGroupId, partition); - } catch (Exception e) { - throw new TaskIdFormatException(string); + } catch (final Exception e) { + throw new TaskIdFormatException(taskIdStr); } } @@ -103,12 +103,8 @@ public int hashCode() { } @Override - public int compareTo(TaskId other) { - return - this.topicGroupId < other.topicGroupId ? -1 : - (this.topicGroupId > other.topicGroupId ? 1 : - (this.partition < other.partition ? -1 : - (this.partition > other.partition ? 1 : - 0))); + public int compareTo(final TaskId other) { + final int compare = Integer.compare(this.topicGroupId, other.topicGroupId); + return compare != 0 ? compare : Integer.compare(this.partition, other.partition); } } diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/AbstractProcessorContext.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/AbstractProcessorContext.java index 0c3fcf201462e..62e0936c80e96 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/AbstractProcessorContext.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/AbstractProcessorContext.java @@ -33,7 +33,7 @@ public abstract class AbstractProcessorContext implements InternalProcessorContext { - static final String NONEXIST_TOPIC = "__null_topic__"; + public static final String NONEXIST_TOPIC = "__null_topic__"; private final TaskId taskId; private final String applicationId; private final StreamsConfig config; diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/AbstractStateManager.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/AbstractStateManager.java index b270e03f2e062..66ddec950c8d9 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/AbstractStateManager.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/AbstractStateManager.java @@ -36,17 +36,18 @@ abstract class AbstractStateManager implements StateManager { static final String CHECKPOINT_FILE_NAME = ".checkpoint"; final File baseDir; - final Map checkpointableOffsets = new HashMap<>(); - + private final boolean eosEnabled; OffsetCheckpoint checkpoint; + final Map checkpointableOffsets = new HashMap<>(); final Map stores = new LinkedHashMap<>(); final Map globalStores = new LinkedHashMap<>(); - AbstractStateManager(final File baseDir) { + AbstractStateManager(final File baseDir, + final boolean eosEnabled) { this.baseDir = baseDir; + this.eosEnabled = eosEnabled; this.checkpoint = new OffsetCheckpoint(new File(baseDir, CHECKPOINT_FILE_NAME)); - } public void reinitializeStateStoresForPartitions(final Logger log, @@ -62,11 +63,14 @@ public void reinitializeStateStoresForPartitions(final Logger log, checkpointableOffsets.remove(topicPartition); storeToBeReinitialized.add(changelogTopicToStore.get(topicPartition.topic())); } - try { - checkpoint.write(checkpointableOffsets); - } catch (final IOException fatalException) { - log.error("Failed to write offset checkpoint file to {} while re-initializing {}: {}", checkpoint, stateStores, fatalException); - throw new StreamsException("Failed to reinitialize global store.", fatalException); + + if (!eosEnabled) { + try { + checkpoint.write(checkpointableOffsets); + } catch (final IOException fatalException) { + log.error("Failed to write offset checkpoint file to {} while re-initializing {}: {}", checkpoint, stateStores, fatalException); + throw new StreamsException("Failed to reinitialize global store.", fatalException); + } } for (final Map.Entry entry : storesCopy.entrySet()) { diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/AbstractTask.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/AbstractTask.java index 02a1a066ab854..94e4c71d9c255 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/AbstractTask.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/AbstractTask.java @@ -137,6 +137,10 @@ public String toString() { return toString(""); } + public boolean isEosEnabled() { + return eosEnabled; + } + /** * Produces a string representation containing useful information about a Task starting with the given indent. * This is useful in debugging scenarios. @@ -167,7 +171,7 @@ public String toString(final String indent) { return sb.toString(); } - protected Map recordCollectorOffsets() { + protected Map activeTaskCheckpointableOffsets() { return Collections.emptyMap(); } @@ -242,7 +246,7 @@ void closeStateManager(final boolean writeCheckpoint) throws ProcessorStateExcep ProcessorStateException exception = null; log.trace("Closing state manager"); try { - stateMgr.close(writeCheckpoint ? recordCollectorOffsets() : null); + stateMgr.close(writeCheckpoint ? activeTaskCheckpointableOffsets() : null); } catch (final ProcessorStateException e) { exception = e; } finally { diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/GlobalStateManagerImpl.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/GlobalStateManagerImpl.java index e8ec5e9fe5f7a..a4ec23d4c4929 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/GlobalStateManagerImpl.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/GlobalStateManagerImpl.java @@ -38,9 +38,11 @@ import java.io.File; import java.io.IOException; +import java.time.Duration; 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; @@ -60,6 +62,8 @@ public class GlobalStateManagerImpl extends AbstractStateManager implements Glob private InternalProcessorContext processorContext; private final int retries; private final long retryBackoffMs; + private final Duration pollTime; + private final Set globalNonPersistentStoresTopics = new HashSet<>(); public GlobalStateManagerImpl(final LogContext logContext, final ProcessorTopology topology, @@ -67,7 +71,15 @@ public GlobalStateManagerImpl(final LogContext logContext, final StateDirectory stateDirectory, final StateRestoreListener stateRestoreListener, final StreamsConfig config) { - super(stateDirectory.globalStateDir()); + super(stateDirectory.globalStateDir(), StreamsConfig.EXACTLY_ONCE.equals(config.getString(StreamsConfig.PROCESSING_GUARANTEE_CONFIG))); + + // Find non persistent store's topics + final Map storeToChangelogTopic = topology.storeToChangelogTopic(); + for (final StateStore store : topology.globalStateStores()) { + if (!store.persistent()) { + globalNonPersistentStoresTopics.add(storeToChangelogTopic.get(store.name())); + } + } this.log = logContext.logger(GlobalStateManagerImpl.class); this.topology = topology; @@ -76,6 +88,7 @@ public GlobalStateManagerImpl(final LogContext logContext, this.stateRestoreListener = stateRestoreListener; this.retries = config.getInt(StreamsConfig.RETRIES_CONFIG); this.retryBackoffMs = config.getLong(StreamsConfig.RETRY_BACKOFF_MS_CONFIG); + this.pollTime = Duration.ofMillis(config.getLong(StreamsConfig.POLL_MS_CONFIG)); } @Override @@ -89,16 +102,16 @@ public Set initialize() { if (!stateDirectory.lockGlobalState()) { throw new LockException(String.format("Failed to lock the global state directory: %s", baseDir)); } - } catch (IOException e) { + } catch (final IOException e) { throw new LockException(String.format("Failed to lock the global state directory: %s", baseDir)); } try { this.checkpointableOffsets.putAll(checkpoint.read()); - } catch (IOException e) { + } catch (final IOException e) { try { stateDirectory.unlockGlobalState(); - } catch (IOException e1) { + } catch (final IOException e1) { log.error("Failed to unlock the global state directory", e); } throw new StreamsException("Failed to read checkpoints for global state globalStores", e); @@ -229,7 +242,7 @@ private List topicPartitionsForStore(final StateStore store) { } final List topicPartitions = new ArrayList<>(); - for (PartitionInfo partition : partitionInfos) { + for (final PartitionInfo partition : partitionInfos) { topicPartitions.add(new TopicPartition(partition.topic(), partition.partition())); } return topicPartitions; @@ -250,8 +263,7 @@ private void restoreState(final StateRestoreCallback stateRestoreCallback, long offset = globalConsumer.position(topicPartition); final Long highWatermark = highWatermarks.get(topicPartition); - BatchingStateRestoreCallback - stateRestoreAdapter = + final BatchingStateRestoreCallback stateRestoreAdapter = (BatchingStateRestoreCallback) ((stateRestoreCallback instanceof BatchingStateRestoreCallback) ? stateRestoreCallback @@ -262,14 +274,14 @@ private void restoreState(final StateRestoreCallback stateRestoreCallback, while (offset < highWatermark) { try { - final ConsumerRecords records = globalConsumer.poll(100); + final ConsumerRecords records = globalConsumer.poll(pollTime); final List> restoreRecords = new ArrayList<>(); - for (ConsumerRecord record : records) { + for (final ConsumerRecord record : records) { if (record.key() != null) { restoreRecords.add(KeyValue.pair(record.key(), record.value())); } - offset = globalConsumer.position(topicPartition); } + offset = globalConsumer.position(topicPartition); stateRestoreAdapter.restoreAll(restoreRecords); stateRestoreListener.onBatchRestored(topicPartition, storeName, offset, restoreRecords.size()); restoreCount += restoreRecords.size(); @@ -291,11 +303,11 @@ private void restoreState(final StateRestoreCallback stateRestoreCallback, @Override public void flush() { log.debug("Flushing all global globalStores registered in the state manager"); - for (StateStore store : this.globalStores.values()) { + for (final StateStore store : this.globalStores.values()) { try { log.trace("Flushing global store={}", store.name()); store.flush(); - } catch (Exception e) { + } catch (final Exception e) { throw new ProcessorStateException(String.format("Failed to flush global state store %s", store.name()), e); } } @@ -313,7 +325,7 @@ public void close(final Map offsets) throws IOException { log.debug("Closing global storage engine {}", entry.getKey()); try { entry.getValue().close(); - } catch (Exception e) { + } catch (final Exception e) { log.error("Failed to close global state store {}", entry.getKey(), e); closeFailed.append("Failed to close global state store:") .append(entry.getKey()) @@ -335,13 +347,22 @@ public void close(final Map offsets) throws IOException { @Override public void checkpoint(final Map offsets) { checkpointableOffsets.putAll(offsets); - if (!checkpointableOffsets.isEmpty()) { - try { - checkpoint.write(checkpointableOffsets); - } catch (IOException e) { - log.warn("Failed to write offset checkpoint file to {} for global stores: {}", checkpoint, e); + + final Map filteredOffsets = new HashMap<>(); + + // Skip non persistent store + for (final Map.Entry topicPartitionOffset : checkpointableOffsets.entrySet()) { + final String topic = topicPartitionOffset.getKey().topic(); + if (!globalNonPersistentStoresTopics.contains(topic)) { + filteredOffsets.put(topicPartitionOffset.getKey(), topicPartitionOffset.getValue()); } } + + try { + checkpoint.write(filteredOffsets); + } catch (final IOException e) { + log.warn("Failed to write offset checkpoint file to {} for global stores: {}", checkpoint, e); + } } @Override diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/GlobalStreamThread.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/GlobalStreamThread.java index 112011f47b8eb..9d529c5455c46 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/GlobalStreamThread.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/GlobalStreamThread.java @@ -36,6 +36,7 @@ import org.slf4j.Logger; import java.io.IOException; +import java.time.Duration; import java.util.Arrays; import java.util.Collections; import java.util.HashSet; @@ -200,7 +201,7 @@ static class StateConsumer { private final Consumer globalConsumer; private final GlobalStateMaintainer stateMaintainer; private final Time time; - private final long pollMs; + private final Duration pollTime; private final long flushInterval; private final Logger log; @@ -210,13 +211,13 @@ static class StateConsumer { final Consumer globalConsumer, final GlobalStateMaintainer stateMaintainer, final Time time, - final long pollMs, + final Duration pollTime, final long flushInterval) { this.log = logContext.logger(getClass()); this.globalConsumer = globalConsumer; this.stateMaintainer = stateMaintainer; this.time = time; - this.pollMs = pollMs; + this.pollTime = pollTime; this.flushInterval = flushInterval; } @@ -235,7 +236,7 @@ void initialize() { void pollAndUpdate() { try { - final ConsumerRecords received = globalConsumer.poll(pollMs); + final ConsumerRecords received = globalConsumer.poll(pollTime); for (final ConsumerRecord record : received) { stateMaintainer.update(record); } @@ -338,8 +339,9 @@ private StateConsumer initialize() { logContext ), time, - config.getLong(StreamsConfig.POLL_MS_CONFIG), - config.getLong(StreamsConfig.COMMIT_INTERVAL_MS_CONFIG)); + Duration.ofMillis(config.getLong(StreamsConfig.POLL_MS_CONFIG)), + config.getLong(StreamsConfig.COMMIT_INTERVAL_MS_CONFIG) + ); stateConsumer.initialize(); return stateConsumer; diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/InternalTopicManager.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/InternalTopicManager.java index 2c2df04590e5d..16568427279c1 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/InternalTopicManager.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/InternalTopicManager.java @@ -52,6 +52,7 @@ public class InternalTopicManager { private final AdminClient adminClient; private final int retries; + private final long retryBackOffMs; public InternalTopicManager(final AdminClient adminClient, final StreamsConfig streamsConfig) { @@ -63,6 +64,7 @@ public InternalTopicManager(final AdminClient adminClient, replicationFactor = streamsConfig.getInt(StreamsConfig.REPLICATION_FACTOR_CONFIG).shortValue(); windowChangeLogAdditionalRetention = streamsConfig.getLong(StreamsConfig.WINDOW_STORE_CHANGE_LOG_ADDITIONAL_RETENTION_MS_CONFIG); retries = new AdminClientConfig(streamsConfig.getAdminConfigs("dummy")).getInt(AdminClientConfig.RETRIES_CONFIG); + retryBackOffMs = new AdminClientConfig(streamsConfig.getAdminConfigs("dummy")).getLong(AdminClientConfig.RETRY_BACKOFF_MS_CONFIG); log.debug("Configs:" + Utils.NL, "\t{} = {}" + Utils.NL, @@ -110,17 +112,22 @@ public void makeReady(final Map topics) { // TODO: KAFKA-6928. should not need retries in the outer caller as it will be retried internally in admin client int remainingRetries = retries; + boolean retryBackOff = false; boolean retry; do { retry = false; final CreateTopicsResult createTopicsResult = adminClient.createTopics(newTopics); - final Set createTopicNames = new HashSet<>(); + final Set createdTopicNames = new HashSet<>(); for (final Map.Entry> createTopicResult : createTopicsResult.values().entrySet()) { try { + if (retryBackOff) { + retryBackOff = false; + Thread.sleep(retryBackOffMs); + } createTopicResult.getValue().get(); - createTopicNames.add(createTopicResult.getKey()); + createdTopicNames.add(createTopicResult.getKey()); } catch (final ExecutionException couldNotCreateTopic) { final Throwable cause = couldNotCreateTopic.getCause(); final String topicName = createTopicResult.getKey(); @@ -130,10 +137,23 @@ public void makeReady(final Map topics) { log.debug("Could not get number of partitions for topic {} due to timeout. " + "Will try again (remaining retries {}).", topicName, remainingRetries - 1); } else if (cause instanceof TopicExistsException) { - createTopicNames.add(createTopicResult.getKey()); - log.info("Topic {} exist already: {}", - topicName, - couldNotCreateTopic.toString()); + // This topic didn't exist earlier, it might be marked for deletion or it might differ + // from the desired setup. It needs re-validation. + final Map existingTopicPartition = getNumPartitions(Collections.singleton(topicName)); + + if (existingTopicPartition.containsKey(topicName) + && validateTopicPartitions(Collections.singleton(topics.get(topicName)), existingTopicPartition).isEmpty()) { + createdTopicNames.add(createTopicResult.getKey()); + log.info("Topic {} exists already and has the right number of partitions: {}", + topicName, + couldNotCreateTopic.toString()); + } else { + retry = true; + retryBackOff = true; + log.info("Could not create topic {}. Topic is probably marked for deletion (number of partitions is unknown).\n" + + "Will retry to create this topic in {} ms (to let broker finish async delete operation first).\n" + + "Error message was: {}", topicName, retryBackOffMs, couldNotCreateTopic.toString()); + } } else { throw new StreamsException(String.format("Could not create topic %s.", topicName), couldNotCreateTopic); @@ -148,7 +168,7 @@ public void makeReady(final Map topics) { if (retry) { final Iterator it = newTopics.iterator(); while (it.hasNext()) { - if (createTopicNames.contains(it.next().name())) { + if (createdTopicNames.contains(it.next().name())) { it.remove(); } } diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/InternalTopologyBuilder.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/InternalTopologyBuilder.java index 7d09031d71386..73db937b7ddae 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/InternalTopologyBuilder.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/InternalTopologyBuilder.java @@ -19,6 +19,7 @@ import org.apache.kafka.common.serialization.Deserializer; import org.apache.kafka.common.serialization.Serializer; import org.apache.kafka.common.utils.Utils; +import org.apache.kafka.streams.StreamsConfig; import org.apache.kafka.streams.Topology; import org.apache.kafka.streams.errors.TopologyException; import org.apache.kafka.streams.processor.ProcessorSupplier; @@ -121,6 +122,9 @@ public class InternalTopologyBuilder { private Map> nodeGroups = null; + // TODO: this is only temporary for 2.0 and should be removed + public final Map storeToSourceChangelogTopic = new HashMap<>(); + public interface StateStoreFactory { Set users(); boolean loggingEnabled(); @@ -438,6 +442,11 @@ public final void addSink(final String name, final String... predecessorNames) { Objects.requireNonNull(name, "name must not be null"); Objects.requireNonNull(topic, "topic must not be null"); + Objects.requireNonNull(predecessorNames, "predecessor names must not be null"); + if (predecessorNames.length == 0) { + throw new TopologyException("Sink " + name + " must have at least one parent"); + } + addSink(name, new StaticTopicNameExtractor(topic), keySerializer, valSerializer, partitioner, predecessorNames); nodeToSinkTopic.put(name, topic); } @@ -450,9 +459,13 @@ public final void addSink(final String name, final String... predecessorNames) { Objects.requireNonNull(name, "name must not be null"); Objects.requireNonNull(topicExtractor, "topic extractor must not be null"); + Objects.requireNonNull(predecessorNames, "predecessor names must not be null"); if (nodeFactories.containsKey(name)) { throw new TopologyException("Processor " + name + " is already added."); } + if (predecessorNames.length == 0) { + throw new TopologyException("Sink " + name + " must have at least one parent"); + } for (final String predecessor : predecessorNames) { Objects.requireNonNull(predecessor, "predecessor name can't be null"); @@ -477,9 +490,13 @@ public final void addProcessor(final String name, final String... predecessorNames) { Objects.requireNonNull(name, "name must not be null"); Objects.requireNonNull(supplier, "supplier must not be null"); + Objects.requireNonNull(predecessorNames, "predecessor names must not be null"); if (nodeFactories.containsKey(name)) { throw new TopologyException("Processor " + name + " is already added."); } + if (predecessorNames.length == 0) { + throw new TopologyException("Processor " + name + " must have at least one parent"); + } for (final String predecessor : predecessorNames) { Objects.requireNonNull(predecessor, "predecessor name must not be null"); @@ -498,8 +515,14 @@ public final void addProcessor(final String name, public final void addStateStore(final StoreBuilder storeBuilder, final String... processorNames) { + addStateStore(storeBuilder, false, processorNames); + } + + public final void addStateStore(final StoreBuilder storeBuilder, + final boolean allowOverride, + final String... processorNames) { Objects.requireNonNull(storeBuilder, "storeBuilder can't be null"); - if (stateFactories.containsKey(storeBuilder.name())) { + if (!allowOverride && stateFactories.containsKey(storeBuilder.name())) { throw new TopologyException("StateStore " + storeBuilder.name() + " is already added."); } @@ -566,38 +589,27 @@ public final void connectProcessorAndStateStores(final String processorName, } } - // TODO: this method is only used by DSL and we might want to refactor this part public final void connectSourceStoreAndTopic(final String sourceStoreName, - final String topic) { + final String topic) { if (storeToChangelogTopic.containsKey(sourceStoreName)) { throw new TopologyException("Source store " + sourceStoreName + " is already added."); } storeToChangelogTopic.put(sourceStoreName, topic); } - // TODO: this method is only used by DSL and we might want to refactor this part - public final void connectProcessors(final String... processorNames) { - if (processorNames.length < 2) { - throw new TopologyException("At least two processors need to participate in the connection."); + public final void markSourceStoreAndTopic(final StoreBuilder storeBuilder, + final String topic) { + if (storeToSourceChangelogTopic.containsKey(storeBuilder)) { + throw new TopologyException("Source store " + storeBuilder.name() + " is already used."); } - - for (final String processorName : processorNames) { - Objects.requireNonNull(processorName, "processor name can't be null"); - if (!nodeFactories.containsKey(processorName)) { - throw new TopologyException("Processor " + processorName + " is not added yet."); - } - } - - nodeGrouper.unite(processorNames[0], Arrays.copyOfRange(processorNames, 1, processorNames.length)); + storeToSourceChangelogTopic.put(storeBuilder, topic); } - // TODO: this method is only used by DSL and we might want to refactor this part public final void addInternalTopic(final String topicName) { Objects.requireNonNull(topicName, "topicName can't be null"); internalTopicNames.add(topicName); } - // TODO: this method is only used by DSL and we might want to refactor this part public final void copartitionSources(final Collection sourceNodes) { copartitionSourceGroups.add(Collections.unmodifiableSet(new HashSet<>(sourceNodes))); } @@ -1015,7 +1027,7 @@ public synchronized Map topicGroups() { if (internalTopicNames.contains(topic)) { // prefix the internal topic name with the application id final String internalTopic = decorateTopic(topic); - repartitionTopics.put(internalTopic, new RepartitionTopicConfig(internalTopic, Collections.emptyMap())); + repartitionTopics.put(internalTopic, new RepartitionTopicConfig(internalTopic, Collections.emptyMap())); sourceTopics.add(internalTopic); } else { sourceTopics.add(topic); @@ -1059,6 +1071,24 @@ public synchronized Map topicGroups() { return Collections.unmodifiableMap(topicGroups); } + // Adjust the generated topology based on the configs. + // Not exposed as public API and should be removed post 2.0 + public void adjust(final StreamsConfig config) { + final boolean enableOptimization20 = config.getString(StreamsConfig.TOPOLOGY_OPTIMIZATION).equals(StreamsConfig.OPTIMIZE); + + if (enableOptimization20) { + for (final Map.Entry entry : storeToSourceChangelogTopic.entrySet()) { + final StoreBuilder storeBuilder = entry.getKey(); + final String topicName = entry.getValue(); + + // update store map to disable logging for this store + storeBuilder.withLoggingDisabled(); + addStateStore(storeBuilder, true); + connectSourceStoreAndTopic(storeBuilder.name(), topicName); + } + } + } + private void setRegexMatchedTopicsToSourceNodes() { if (subscriptionUpdates.hasUpdates()) { for (final Map.Entry stringPatternEntry : nodeToSourcePatterns.entrySet()) { @@ -1323,7 +1353,7 @@ private void describeSubtopology(final TopologyDescription description, description.addSubtopology(new Subtopology( subtopologyId, - new HashSet(nodesByName.values()))); + new HashSet<>(nodesByName.values()))); } public final static class GlobalStore implements TopologyDescription.GlobalStore { @@ -1714,10 +1744,10 @@ public Set globalStores() { public String toString() { final StringBuilder sb = new StringBuilder(); sb.append("Topologies:\n "); - final TopologyDescription.Subtopology[] sortedSubtopologies = - subtopologies.descendingSet().toArray(new TopologyDescription.Subtopology[subtopologies.size()]); - final TopologyDescription.GlobalStore[] sortedGlobalStores = - globalStores.descendingSet().toArray(new TopologyDescription.GlobalStore[globalStores.size()]); + final TopologyDescription.Subtopology[] sortedSubtopologies = + subtopologies.descendingSet().toArray(new Subtopology[0]); + final TopologyDescription.GlobalStore[] sortedGlobalStores = + globalStores.descendingSet().toArray(new GlobalStore[0]); int expectedId = 0; int subtopologiesIndex = sortedSubtopologies.length - 1; int globalStoresIndex = sortedGlobalStores.length - 1; @@ -1818,7 +1848,7 @@ public String toString() { public void updateSubscribedTopics(final Set topics, final String logPrefix) { final SubscriptionUpdates subscriptionUpdates = new SubscriptionUpdates(); - log.debug("{}found {} topics possibly matching regex", topics, logPrefix); + log.debug("{}found {} topics possibly matching regex", logPrefix, topics); // update the topic groups with the returned subscription set for regex pattern subscriptions subscriptionUpdates.updateTopics(topics); updateSubscriptions(subscriptionUpdates, logPrefix); 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 index a539a1bcda0a5..6300553a9115f 100644 --- 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 @@ -105,18 +105,22 @@ public void forward(final K key, final V value, final String childName) { @SuppressWarnings("unchecked") @Override public void forward(final K key, final V value, final To to) { - toInternal.update(to); - if (toInternal.hasTimestamp()) { - recordContext.setTimestamp(toInternal.timestamp()); - } final ProcessorNode previousNode = currentNode(); + final long currentTimestamp = recordContext.timestamp; + try { + toInternal.update(to); + if (toInternal.hasTimestamp()) { + recordContext.setTimestamp(toInternal.timestamp()); + } + final List> children = (List>) currentNode().children(); final String sendTo = toInternal.child(); if (sendTo != null) { final ProcessorNode child = currentNode().getChild(sendTo); if (child == null) { - throw new StreamsException("Unknown processor name: " + sendTo); + throw new StreamsException("Unknown downstream node: " + sendTo + " either does not exist or is not" + + " connected to this processor."); } forward(child, key, value); } else { @@ -130,6 +134,7 @@ public void forward(final K key, final V value, final To to) { } } } finally { + recordContext.timestamp = currentTimestamp; setCurrentNode(previousNode); } } 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 index e7a23bd4b5f5d..afb56c1ac1b19 100644 --- 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 @@ -46,7 +46,7 @@ public class ProcessorStateManager extends AbstractStateManager { private final boolean isStandby; private final ChangelogReader changelogReader; private final Map offsetLimits; - private final Map restoredOffsets; + private final Map standbyRestoredOffsets; private final Map restoreCallbacks; // used for standby tasks, keyed by state topic name private final Map storeToChangelogTopic; private final List changelogPartitions = new ArrayList<>(); @@ -67,7 +67,7 @@ public ProcessorStateManager(final TaskId taskId, final ChangelogReader changelogReader, final boolean eosEnabled, final LogContext logContext) throws IOException { - super(stateDirectory.directoryForTask(taskId)); + super(stateDirectory.directoryForTask(taskId), eosEnabled); this.log = logContext.logger(ProcessorStateManager.class); this.taskId = taskId; @@ -79,14 +79,13 @@ public ProcessorStateManager(final TaskId taskId, partitionForTopic.put(source.topic(), source); } offsetLimits = new HashMap<>(); - restoredOffsets = new HashMap<>(); + standbyRestoredOffsets = new HashMap<>(); this.isStandby = isStandby; - restoreCallbacks = isStandby ? new HashMap() : null; + restoreCallbacks = isStandby ? new HashMap<>() : null; this.storeToChangelogTopic = storeToChangelogTopic; // load the checkpoint information checkpointableOffsets.putAll(checkpoint.read()); - if (eosEnabled) { // delete the checkpoint file after finish loading its stored offsets checkpoint.delete(); @@ -169,11 +168,7 @@ public Map checkpointed() { final int partition = getPartition(topicName); final TopicPartition storePartition = new TopicPartition(topicName, partition); - if (checkpointableOffsets.containsKey(storePartition)) { - partitionsAndOffsets.put(storePartition, checkpointableOffsets.get(storePartition)); - } else { - partitionsAndOffsets.put(storePartition, -1L); - } + partitionsAndOffsets.put(storePartition, checkpointableOffsets.getOrDefault(storePartition, -1L)); } return partitionsAndOffsets; } @@ -212,7 +207,7 @@ List> updateStandbyStates(final TopicPartition st } // record the restored offset for its change log partition - restoredOffsets.put(storePartition, lastOffset + 1); + standbyRestoredOffsets.put(storePartition, lastOffset + 1); return remainingRecords; } @@ -293,8 +288,8 @@ public void close(final Map ackedOffsets) throws Processor // write the checkpoint @Override - public void checkpoint(final Map ackedOffsets) { - checkpointableOffsets.putAll(changelogReader.restoredOffsets()); + public void checkpoint(final Map checkpointableOffsets) { + this.checkpointableOffsets.putAll(changelogReader.restoredOffsets()); for (final StateStore store : stores.values()) { final String storeName = store.name(); // only checkpoint the offset to the offsets file if @@ -302,11 +297,11 @@ public void checkpoint(final Map ackedOffsets) { if (store.persistent() && storeToChangelogTopic.containsKey(storeName)) { final String changelogTopic = storeToChangelogTopic.get(storeName); final TopicPartition topicPartition = new TopicPartition(changelogTopic, getPartition(storeName)); - if (ackedOffsets.containsKey(topicPartition)) { + if (checkpointableOffsets.containsKey(topicPartition)) { // store the last offset + 1 (the log position after restoration) - checkpointableOffsets.put(topicPartition, ackedOffsets.get(topicPartition) + 1); - } else if (restoredOffsets.containsKey(topicPartition)) { - checkpointableOffsets.put(topicPartition, restoredOffsets.get(topicPartition)); + this.checkpointableOffsets.put(topicPartition, checkpointableOffsets.get(topicPartition) + 1); + } else if (standbyRestoredOffsets.containsKey(topicPartition)) { + this.checkpointableOffsets.put(topicPartition, standbyRestoredOffsets.get(topicPartition)); } } } @@ -315,9 +310,9 @@ public void checkpoint(final Map ackedOffsets) { checkpoint = new OffsetCheckpoint(new File(baseDir, CHECKPOINT_FILE_NAME)); } - log.trace("Writing checkpoint: {}", checkpointableOffsets); + log.trace("Writing checkpoint: {}", this.checkpointableOffsets); try { - checkpoint.write(checkpointableOffsets); + checkpoint.write(this.checkpointableOffsets); } catch (final IOException e) { log.warn("Failed to write offset checkpoint file to {}: {}", checkpoint, e); } @@ -340,7 +335,7 @@ public StateStore getGlobalStore(final String name) { return globalStores.get(name); } - private BatchingStateRestoreCallback getBatchingRestoreCallback(StateRestoreCallback callback) { + private BatchingStateRestoreCallback getBatchingRestoreCallback(final StateRestoreCallback callback) { if (callback instanceof BatchingStateRestoreCallback) { return (BatchingStateRestoreCallback) callback; } 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 index bf10da2b5e7fa..d31d5963e4495 100644 --- 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 @@ -44,6 +44,12 @@ void send(final String topic, final Serializer valueSerializer, final StreamPartitioner partitioner); + /** + * Initialize the collector with a producer. + * @param producer the producer that should be used by this collector + */ + void init(final Producer producer); + /** * Flush the internal {@link Producer}. */ @@ -57,7 +63,7 @@ void send(final String topic, /** * The last acked offsets from the internal {@link Producer}. * - * @return the map from TopicPartition to offset + * @return an immutable map from TopicPartition to offset */ Map offsets(); diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/RecordCollectorImpl.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/RecordCollectorImpl.java index d753648eedee0..05b8c3cdfab1d 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/RecordCollectorImpl.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/RecordCollectorImpl.java @@ -16,6 +16,7 @@ */ package org.apache.kafka.streams.processor.internals; +import java.util.Collections; import org.apache.kafka.clients.producer.Callback; import org.apache.kafka.clients.producer.Producer; import org.apache.kafka.clients.producer.ProducerRecord; @@ -51,7 +52,7 @@ public class RecordCollectorImpl implements RecordCollector { private final Logger log; private final String logPrefix; private final Sensor skippedRecordsSensor; - private final Producer producer; + private Producer producer; private final Map offsets; private final ProductionExceptionHandler productionExceptionHandler; @@ -61,12 +62,10 @@ public class RecordCollectorImpl implements RecordCollector { private final static String PARAMETER_HINT = "\nYou can increase producer parameter `retries` and `retry.backoff.ms` to avoid this error."; private volatile KafkaException sendException; - public RecordCollectorImpl(final Producer producer, - final String streamTaskId, + public RecordCollectorImpl(final String streamTaskId, final LogContext logContext, final ProductionExceptionHandler productionExceptionHandler, final Sensor skippedRecordsSensor) { - this.producer = producer; this.offsets = new HashMap<>(); this.logPrefix = String.format("task [%s] ", streamTaskId); this.log = logContext.logger(getClass()); @@ -74,6 +73,11 @@ public RecordCollectorImpl(final Producer producer, this.skippedRecordsSensor = skippedRecordsSensor; } + @Override + public void init(final Producer producer) { + this.producer = producer; + } + @Override public void send(final String topic, final K key, @@ -201,24 +205,41 @@ public void onCompletion(final RecordMetadata metadata, } }); } catch (final TimeoutException e) { - log.error("Timeout exception caught when sending record to topic {}. " + - "This might happen if the producer cannot send data to the Kafka cluster and thus, " + - "its internal buffer fills up. " + - "You can increase producer parameter `max.block.ms` to increase this timeout.", topic); - throw new StreamsException(String.format("%sFailed to send record to topic %s due to timeout.", logPrefix, topic)); - } catch (final Exception uncaughtException) { + log.error( + "Timeout exception caught when sending record to topic {}. " + + "This might happen if the producer cannot send data to the Kafka cluster and thus, " + + "its internal buffer fills up. " + + "This can also happen if the broker is slow to respond, if the network connection to " + + "the broker was interrupted, or if similar circumstances arise. " + + "You can increase producer parameter `max.block.ms` to increase this timeout.", + topic, + e + ); throw new StreamsException( - String.format( - EXCEPTION_MESSAGE, - logPrefix, - "an error caught", - key, - value, - timestamp, - topic, - uncaughtException.toString() - ), - uncaughtException); + String.format("%sFailed to send record to topic %s due to timeout.", logPrefix, topic), + e + ); + } catch (final Exception uncaughtException) { + if (uncaughtException instanceof KafkaException && + uncaughtException.getCause() instanceof ProducerFencedException) { + final KafkaException kafkaException = (KafkaException) uncaughtException; + // producer.send() call may throw a KafkaException which wraps a FencedException, + // in this case we should throw its wrapped inner cause so that it can be captured and re-wrapped as TaskMigrationException + throw (ProducerFencedException) kafkaException.getCause(); + } else { + throw new StreamsException( + String.format( + EXCEPTION_MESSAGE, + logPrefix, + "an error caught", + key, + value, + timestamp, + topic, + uncaughtException.toString() + ), + uncaughtException); + } } } @@ -238,13 +259,16 @@ public void flush() { @Override public void close() { log.debug("Closing producer"); - producer.close(); + if (producer != null) { + producer.close(); + producer = null; + } checkForException(); } @Override public Map offsets() { - return offsets; + return Collections.unmodifiableMap(offsets); } // for testing only diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/StandbyContextImpl.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/StandbyContextImpl.java index 5c278c9c75403..f920305c9dabc 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/StandbyContextImpl.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/StandbyContextImpl.java @@ -16,6 +16,7 @@ */ package org.apache.kafka.streams.processor.internals; +import org.apache.kafka.clients.producer.Producer; import org.apache.kafka.common.TopicPartition; import org.apache.kafka.common.header.Headers; import org.apache.kafka.common.serialization.Serializer; @@ -58,6 +59,9 @@ public void send(final String topic, final Serializer valueSerializer, final StreamPartitioner partitioner) {} + @Override + public void init(final Producer producer) {} + @Override public void flush() {} diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/StateDirectory.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/StateDirectory.java index c33ade6f36e85..7623c66cd3b71 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/StateDirectory.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/StateDirectory.java @@ -91,7 +91,7 @@ public StateDirectory(final StreamsConfig config, * @return directory for the {@link TaskId} * @throws ProcessorStateException if the task directory does not exists and could not be created */ - File directoryForTask(final TaskId taskId) { + public File directoryForTask(final TaskId taskId) { final File taskDir = new File(stateDir, taskId.toString()); if (!taskDir.exists() && !taskDir.mkdir()) { throw new ProcessorStateException( diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/StateRestorer.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/StateRestorer.java index 33dce9e755814..3bbf42ead272b 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/StateRestorer.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/StateRestorer.java @@ -26,13 +26,13 @@ public class StateRestorer { static final int NO_CHECKPOINT = -1; - private final Long checkpoint; private final long offsetLimit; private final boolean persistent; private final String storeName; private final TopicPartition partition; private final CompositeRestoreListener compositeRestoreListener; + private long checkpointOffset; private long restoredOffset; private long startingOffset; private long endingOffset; @@ -45,7 +45,7 @@ public class StateRestorer { final String storeName) { this.partition = partition; this.compositeRestoreListener = compositeRestoreListener; - this.checkpoint = checkpoint; + this.checkpointOffset = checkpoint == null ? NO_CHECKPOINT : checkpoint; this.offsetLimit = offsetLimit; this.persistent = persistent; this.storeName = storeName; @@ -55,8 +55,16 @@ public TopicPartition partition() { return partition; } + public String storeName() { + return storeName; + } + long checkpoint() { - return checkpoint == null ? NO_CHECKPOINT : checkpoint; + return checkpointOffset; + } + + void setCheckpointOffset(final long checkpointOffset) { + this.checkpointOffset = checkpointOffset; } void restoreStarted() { diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/StoreChangelogReader.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/StoreChangelogReader.java index af5ff4728f5e3..8729b842edb3c 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/StoreChangelogReader.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/StoreChangelogReader.java @@ -29,6 +29,7 @@ import org.apache.kafka.streams.processor.StateRestoreListener; import org.slf4j.Logger; +import java.time.Duration; import java.util.ArrayList; import java.util.Collection; import java.util.Collections; @@ -47,31 +48,36 @@ public class StoreChangelogReader implements ChangelogReader { private final Map endOffsets = new HashMap<>(); private final Map> partitionInfo = new HashMap<>(); private final Map stateRestorers = new HashMap<>(); - private final Map needsRestoring = new HashMap<>(); - private final Map needsInitializing = new HashMap<>(); - private Map updatedEndOffsets = new HashMap<>(); + private final Set needsRestoring = new HashSet<>(); + private final Set needsInitializing = new HashSet<>(); + private final Set completedRestorers = new HashSet<>(); + private final Duration pollTime; public StoreChangelogReader(final Consumer restoreConsumer, + final Duration pollTime, final StateRestoreListener userStateRestoreListener, final LogContext logContext) { this.restoreConsumer = restoreConsumer; + this.pollTime = pollTime; this.log = logContext.logger(getClass()); this.userStateRestoreListener = userStateRestoreListener; } @Override public void register(final StateRestorer restorer) { - restorer.setUserRestoreListener(userStateRestoreListener); - stateRestorers.put(restorer.partition(), restorer); - needsInitializing.put(restorer.partition(), restorer); + if (!stateRestorers.containsKey(restorer.partition())) { + restorer.setUserRestoreListener(userStateRestoreListener); + stateRestorers.put(restorer.partition(), restorer); + + log.trace("Added restorer for changelog {}", restorer.partition()); + } + + needsInitializing.add(restorer.partition()); } public Collection restore(final RestoringTasks active) { if (!needsInitializing.isEmpty()) { - initialize(); - final Set remainingPartitions = new HashSet<>(needsRestoring.keySet()); - remainingPartitions.removeAll(updatedEndOffsets.keySet()); - updatedEndOffsets.putAll(restoreConsumer.endOffsets(remainingPartitions)); + initialize(active); } if (needsRestoring.isEmpty()) { @@ -80,17 +86,16 @@ public Collection restore(final RestoringTasks active) { } try { - final ConsumerRecords records = restoreConsumer.poll(10); - final Iterator iterator = needsRestoring.keySet().iterator(); - while (iterator.hasNext()) { - final TopicPartition partition = iterator.next(); + final ConsumerRecords records = restoreConsumer.poll(pollTime); + + for (final TopicPartition partition : needsRestoring) { final StateRestorer restorer = stateRestorers.get(partition); - final long pos = processNext(records.records(partition), restorer, updatedEndOffsets.get(partition)); + final long pos = processNext(records.records(partition), restorer, endOffsets.get(partition)); restorer.setRestoredOffset(pos); - if (restorer.hasCompleted(pos, updatedEndOffsets.get(partition))) { + if (restorer.hasCompleted(pos, endOffsets.get(partition))) { restorer.restoreDone(); - updatedEndOffsets.remove(partition); - iterator.remove(); + endOffsets.remove(partition); + completedRestorers.add(partition); } } } catch (final InvalidOffsetException recoverableException) { @@ -98,12 +103,20 @@ public Collection restore(final RestoringTasks active) { final Set partitions = recoverableException.partitions(); for (final TopicPartition partition : partitions) { final StreamTask task = active.restoringTaskFor(partition); - log.info("Reinitializing StreamTask {}", task); + log.info("Reinitializing StreamTask {} for changelog {}", task, partition); + + needsInitializing.remove(partition); + needsRestoring.remove(partition); + + final StateRestorer restorer = stateRestorers.get(partition); + restorer.setCheckpointOffset(StateRestorer.NO_CHECKPOINT); task.reinitializeStateStoresForPartitions(recoverableException.partitions()); } restoreConsumer.seekToBeginning(partitions); } + needsRestoring.removeAll(completedRestorers); + if (needsRestoring.isEmpty()) { restoreConsumer.unsubscribe(); } @@ -111,7 +124,7 @@ public Collection restore(final RestoringTasks active) { return completed(); } - private void initialize() { + private void initialize(final RestoringTasks active) { if (!restoreConsumer.subscription().isEmpty()) { throw new StreamsException("Restore consumer should not be subscribed to any topics (" + restoreConsumer.subscription() + ")"); } @@ -120,25 +133,24 @@ private void initialize() { // the needsInitializing map is not empty, meaning we do not know the metadata for some of them yet refreshChangelogInfo(); - Map initializable = new HashMap<>(); - for (Map.Entry entry : needsInitializing.entrySet()) { - final TopicPartition topicPartition = entry.getKey(); + final Set initializable = new HashSet<>(); + for (final TopicPartition topicPartition : needsInitializing) { if (hasPartition(topicPartition)) { - initializable.put(entry.getKey(), entry.getValue()); + initializable.add(topicPartition); } } // try to fetch end offsets for the initializable restorers and remove any partitions // where we already have all of the data try { - endOffsets.putAll(restoreConsumer.endOffsets(initializable.keySet())); + endOffsets.putAll(restoreConsumer.endOffsets(initializable)); } catch (final TimeoutException e) { // if timeout exception gets thrown we just give up this time and retry in the next run loop log.debug("Could not fetch end offset for {}; will fall back to partition by partition fetching", initializable); return; } - final Iterator iter = initializable.keySet().iterator(); + final Iterator iter = initializable.iterator(); while (iter.hasNext()) { final TopicPartition topicPartition = iter.next(); final Long endOffset = endOffsets.get(topicPartition); @@ -146,13 +158,15 @@ private void initialize() { // offset should not be null; but since the consumer API does not guarantee it // we add this check just in case if (endOffset != null) { - final StateRestorer restorer = needsInitializing.get(topicPartition); + final StateRestorer restorer = stateRestorers.get(topicPartition); if (restorer.checkpoint() >= endOffset) { restorer.setRestoredOffset(restorer.checkpoint()); iter.remove(); + completedRestorers.add(topicPartition); } else if (restorer.offsetLimit() == 0 || endOffset == 0) { restorer.setRestoredOffset(0); iter.remove(); + completedRestorers.add(topicPartition); } else { restorer.setEndingOffset(endOffset); } @@ -165,42 +179,63 @@ private void initialize() { // set up restorer for those initializable if (!initializable.isEmpty()) { - startRestoration(initializable); + startRestoration(initializable, active); } } - private void startRestoration(final Map initialized) { - log.debug("Start restoring state stores from changelog topics {}", initialized.keySet()); + private void startRestoration(final Set initialized, + final RestoringTasks active) { + log.debug("Start restoring state stores from changelog topics {}", initialized); final Set assignment = new HashSet<>(restoreConsumer.assignment()); - assignment.addAll(initialized.keySet()); + assignment.addAll(initialized); restoreConsumer.assign(assignment); final List needsPositionUpdate = new ArrayList<>(); - for (final StateRestorer restorer : initialized.values()) { + + for (final TopicPartition partition : initialized) { + final StateRestorer restorer = stateRestorers.get(partition); if (restorer.checkpoint() != StateRestorer.NO_CHECKPOINT) { - restoreConsumer.seek(restorer.partition(), restorer.checkpoint()); - logRestoreOffsets(restorer.partition(), - restorer.checkpoint(), - endOffsets.get(restorer.partition())); - restorer.setStartingOffset(restoreConsumer.position(restorer.partition())); + restoreConsumer.seek(partition, restorer.checkpoint()); + logRestoreOffsets(partition, + restorer.checkpoint(), + endOffsets.get(partition)); + restorer.setStartingOffset(restoreConsumer.position(partition)); restorer.restoreStarted(); } else { - restoreConsumer.seekToBeginning(Collections.singletonList(restorer.partition())); + restoreConsumer.seekToBeginning(Collections.singletonList(partition)); needsPositionUpdate.add(restorer); } } for (final StateRestorer restorer : needsPositionUpdate) { - final long position = restoreConsumer.position(restorer.partition()); - logRestoreOffsets(restorer.partition(), - position, - endOffsets.get(restorer.partition())); - restorer.setStartingOffset(position); - restorer.restoreStarted(); + final TopicPartition partition = restorer.partition(); + + // If checkpoint does not exist it means the task was not shutdown gracefully before; + // and in this case if EOS is turned on we should wipe out the state and re-initialize the task + final StreamTask task = active.restoringTaskFor(partition); + if (task.isEosEnabled()) { + log.info("No checkpoint found for task {} state store {} changelog {} with EOS turned on. " + + "Reinitializing the task and restore its state from the beginning.", task.id, restorer.storeName(), partition); + + needsInitializing.remove(partition); + initialized.remove(partition); + restorer.setCheckpointOffset(restoreConsumer.position(partition)); + + task.reinitializeStateStoresForPartitions(Collections.singleton(partition)); + } else { + log.info("Restoring task {}'s state store {} from beginning of the changelog {} ", task.id, restorer.storeName(), partition); + + final long position = restoreConsumer.position(restorer.partition()); + logRestoreOffsets(restorer.partition(), + position, + endOffsets.get(restorer.partition())); + restorer.setStartingOffset(position); + restorer.restoreStarted(); + } } - needsRestoring.putAll(initialized); + needsRestoring.addAll(initialized); } private void logRestoreOffsets(final TopicPartition partition, @@ -213,10 +248,7 @@ private void logRestoreOffsets(final TopicPartition partition, } private Collection completed() { - final Set completed = new HashSet<>(stateRestorers.keySet()); - completed.removeAll(needsRestoring.keySet()); - log.trace("The set of restoration completed partitions so far: {}", completed); - return completed; + return completedRestorers; } private void refreshChangelogInfo() { @@ -253,7 +285,7 @@ private long processNext(final List> records, final Long endOffset) { final List> restoreRecords = new ArrayList<>(); long nextPosition = -1; - int numberRecords = records.size(); + final int numberRecords = records.size(); int numberRestored = 0; long lastRestoredOffset = -1; for (final ConsumerRecord record : records) { @@ -280,6 +312,9 @@ private long processNext(final List> records, if (!restoreRecords.isEmpty()) { restorer.restore(restoreRecords); restorer.restoreBatchCompleted(lastRestoredOffset, records.size()); + + log.trace("Restored from {} to {} with {} records, ending offset is {}, next starting position is {}", + restorer.partition(), restorer.storeName(), records.size(), lastRestoredOffset, nextPosition); } return nextPosition; @@ -299,6 +334,7 @@ private boolean hasPartition(final TopicPartition topicPartition) { return true; } } + return false; } } 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 index e2be3e2917214..053ceb32532be 100644 --- 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 @@ -25,6 +25,7 @@ import org.apache.kafka.common.MetricName; import org.apache.kafka.common.TopicPartition; import org.apache.kafka.common.errors.ProducerFencedException; +import org.apache.kafka.common.errors.TimeoutException; import org.apache.kafka.common.metrics.Sensor; import org.apache.kafka.common.metrics.stats.Avg; import org.apache.kafka.common.metrics.stats.Count; @@ -66,7 +67,8 @@ public class StreamTask extends AbstractTask implements ProcessorNodePunctuator private final Map consumedOffsets; private final RecordCollector recordCollector; - private final Producer producer; + private final ProducerSupplier producerSupplier; + private Producer producer; private final int maxBufferedSize; private boolean commitRequested = false; @@ -108,7 +110,7 @@ protected static final class TaskMetrics { // add the operation metrics with additional tags final Map tagMap = metrics.tagMap("task-id", taskName); - taskCommitTimeSensor = metrics.taskLevelSensor("commit", taskName, Sensor.RecordingLevel.DEBUG, parent); + taskCommitTimeSensor = metrics.taskLevelSensor(taskName, "commit", Sensor.RecordingLevel.DEBUG, parent); taskCommitTimeSensor.add( new MetricName("commit-latency-avg", group, "The average latency of commit operation.", tagMap), new Avg() @@ -132,6 +134,10 @@ void removeAllSensors() { } } + public interface ProducerSupplier { + Producer get(); + } + public StreamTask(final TaskId id, final Collection partitions, final ProcessorTopology topology, @@ -142,8 +148,8 @@ public StreamTask(final TaskId id, final StateDirectory stateDirectory, final ThreadCache cache, final Time time, - final Producer producer) { - this(id, partitions, topology, consumer, changelogReader, config, metrics, stateDirectory, cache, time, producer, null); + final ProducerSupplier producerSupplier) { + this(id, partitions, topology, consumer, changelogReader, config, metrics, stateDirectory, cache, time, producerSupplier, null); } public StreamTask(final TaskId id, @@ -156,19 +162,19 @@ public StreamTask(final TaskId id, final StateDirectory stateDirectory, final ThreadCache cache, final Time time, - final Producer producer, + final ProducerSupplier producerSupplier, final RecordCollector recordCollector) { super(id, partitions, topology, consumer, changelogReader, false, stateDirectory, config); this.time = time; - this.producer = producer; + this.producerSupplier = producerSupplier; + this.producer = producerSupplier.get(); this.taskMetrics = new TaskMetrics(id, metrics); final ProductionExceptionHandler productionExceptionHandler = config.defaultProductionExceptionHandler(); if (recordCollector == null) { this.recordCollector = new RecordCollectorImpl( - producer, id.toString(), logContext, productionExceptionHandler, @@ -177,6 +183,8 @@ public StreamTask(final TaskId id, } else { this.recordCollector = recordCollector; } + this.recordCollector.init(this.producer); + streamTimePunctuationQueue = new PunctuationQueue(); systemTimePunctuationQueue = new PunctuationQueue(); maxBufferedSize = config.getInt(StreamsConfig.BUFFERED_RECORDS_PER_PARTITION_CONFIG); @@ -215,7 +223,7 @@ public StreamTask(final TaskId id, // initialize transactions if eos is turned on, which will block if the previous transaction has not // completed yet; do not start the first transaction until the topology has been initialized later if (eosEnabled) { - this.producer.initTransactions(); + initializeTransactions(); } } @@ -257,8 +265,15 @@ public void initializeTopology() { */ @Override public void resume() { - // nothing to do; new transaction will be started only after topology is initialized log.debug("Resuming"); + if (eosEnabled) { + if (producer != null) { + throw new IllegalStateException("Task producer should be null."); + } + producer = producerSupplier.get(); + initializeTransactions(); + recordCollector.init(producer); + } } /** @@ -380,7 +395,7 @@ void commit(final boolean startNewTransaction) { flushState(); if (!eosEnabled) { - stateMgr.checkpoint(recordCollectorOffsets()); + stateMgr.checkpoint(activeTaskCheckpointableOffsets()); } commitOffsets(startNewTransaction); @@ -391,8 +406,14 @@ void commit(final boolean startNewTransaction) { } @Override - protected Map recordCollectorOffsets() { - return recordCollector.offsets(); + protected Map activeTaskCheckpointableOffsets() { + final Map checkpointableOffsets = + new HashMap<>(recordCollector.offsets()); + for (final Map.Entry entry : consumedOffsets.entrySet()) { + checkpointableOffsets.putIfAbsent(entry.getKey(), entry.getValue()); + } + + return checkpointableOffsets; } @Override @@ -424,19 +445,19 @@ private void commitOffsets(final boolean startNewTransaction) { if (eosEnabled) { producer.sendOffsetsToTransaction(consumedOffsetsAndMetadata, applicationId); - producer.commitTransaction(); - transactionInFlight = false; - if (startNewTransaction) { - producer.beginTransaction(); - transactionInFlight = true; - } } else { consumer.commitSync(consumedOffsetsAndMetadata); } commitOffsetNeeded = false; - } else if (eosEnabled && !startNewTransaction && transactionInFlight) { // need to make sure to commit txn for suspend case + } + + if (eosEnabled) { producer.commitTransaction(); transactionInFlight = false; + if (startNewTransaction) { + producer.beginTransaction(); + transactionInFlight = true; + } } } catch (final CommitFailedException | ProducerFencedException fatal) { throw new TaskMigratedException(this, fatal); @@ -483,7 +504,7 @@ private void initTopology() { @Override public void suspend() { log.debug("Suspending"); - suspend(true); + suspend(true, false); } /** @@ -499,10 +520,66 @@ public void suspend() { * or if the task producer got fenced (EOS) */ // visible for testing - void suspend(final boolean clean) { - closeTopology(); // should we call this only on clean suspend? + void suspend(final boolean clean, + final boolean isZombie) { + try { + closeTopology(); // should we call this only on clean suspend? + } catch (final RuntimeException fatal) { + if (clean) { + throw fatal; + } + } + if (clean) { - commit(false); + TaskMigratedException taskMigratedException = null; + try { + commit(false); + } finally { + if (eosEnabled) { + try { + recordCollector.close(); + } catch (final ProducerFencedException e) { + taskMigratedException = new TaskMigratedException(this, e); + } finally { + producer = null; + } + } + } + if (taskMigratedException != null) { + throw taskMigratedException; + } + } else { + maybeAbortTransactionAndCloseRecordCollector(isZombie); + } + } + + private void maybeAbortTransactionAndCloseRecordCollector(final boolean isZombie) { + if (eosEnabled && !isZombie) { + try { + if (transactionInFlight) { + producer.abortTransaction(); + } + transactionInFlight = false; + } catch (final ProducerFencedException ignore) { + /* TODO + * this should actually never happen atm as we guard the call to #abortTransaction + * -> the reason for the guard is a "bug" in the Producer -- it throws IllegalStateException + * instead of ProducerFencedException atm. We can remove the isZombie flag after KAFKA-5604 got + * fixed and fall-back to this catch-and-swallow code + */ + + // can be ignored: transaction got already aborted by brokers/transactional-coordinator if this happens + } + } + + if (eosEnabled) { + try { + recordCollector.close(); + } catch (final Throwable e) { + log.error("Failed to close producer due to the following error:", e); + } finally { + producer = null; + } } } @@ -547,37 +624,8 @@ public void closeSuspended(boolean clean, log.error("Could not close state manager due to the following error:", e); } - try { - partitionGroup.close(); - taskMetrics.removeAllSensors(); - } finally { - if (eosEnabled) { - if (!clean) { - try { - if (!isZombie && transactionInFlight) { - producer.abortTransaction(); - } - transactionInFlight = false; - } catch (final ProducerFencedException ignore) { - /* TODO - * this should actually never happen atm as we guard the call to #abortTransaction - * -> the reason for the guard is a "bug" in the Producer -- it throws IllegalStateException - * instead of ProducerFencedException atm. We can remove the isZombie flag after KAFKA-5604 got - * fixed and fall-back to this catch-and-swallow code - */ - - // can be ignored: transaction got already aborted by brokers/transactional-coordinator if this happens - } - } - try { - if (!isZombie) { - recordCollector.close(); - } - } catch (final Throwable e) { - log.error("Failed to close producer due to the following error:", e); - } - } - } + partitionGroup.close(); + taskMetrics.removeAllSensors(); if (firstException != null) { throw firstException; @@ -586,7 +634,7 @@ public void closeSuspended(boolean clean, /** *

                        -     * - {@link #suspend(boolean) suspend(clean)}
                        +     * - {@link #suspend(boolean, boolean) suspend(clean)}
                              *   - close topology
                              *   - if (clean) {@link #commit()}
                              *     - flush state and producer
                        @@ -609,7 +657,7 @@ public void close(boolean clean,
                         
                                 RuntimeException firstException = null;
                                 try {
                        -            suspend(clean);
                        +            suspend(clean, isZombie);
                                 } catch (final RuntimeException e) {
                                     clean = false;
                                     firstException = e;
                        @@ -752,4 +800,23 @@ RecordCollector recordCollector() {
                             Producer getProducer() {
                                 return producer;
                             }
                        +
                        +    private void initializeTransactions() {
                        +        try {
                        +            producer.initTransactions();
                        +        } catch (final TimeoutException retriable) {
                        +            log.error(
                        +                "Timeout exception caught when initializing transactions for task {}. " +
                        +                    "This might happen if the broker is slow to respond, if the network connection to " +
                        +                    "the broker was interrupted, or if similar circumstances arise. " +
                        +                    "You can increase producer parameter `max.block.ms` to increase this timeout.",
                        +                id,
                        +                retriable
                        +            );
                        +            throw new StreamsException(
                        +                format("%sFailed to initialize task %s due to timeout.", logPrefix, id),
                        +                retriable
                        +            );
                        +        }
                        +    }
                         }
                        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
                        index e72c4a5de9405..038881f2f2512 100644
                        --- 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
                        @@ -50,6 +50,7 @@
                         import org.apache.kafka.streams.state.internals.ThreadCache;
                         import org.slf4j.Logger;
                         
                        +import java.time.Duration;
                         import java.util.ArrayList;
                         import java.util.Arrays;
                         import java.util.Collection;
                        @@ -212,7 +213,7 @@ State setState(final State newState) {
                                     if (newState == State.RUNNING) {
                                         updateThreadMetadata(taskManager.activeTasks(), taskManager.standbyTasks());
                                     } else {
                        -                updateThreadMetadata(Collections.emptyMap(), Collections.emptyMap());
                        +                updateThreadMetadata(Collections.emptyMap(), Collections.emptyMap());
                                     }
                                 }
                         
                        @@ -432,7 +433,7 @@ StreamTask createTask(final Consumer consumer,
                                         stateDirectory,
                                         cache,
                                         time,
                        -                createProducer(taskId)
                        +                () -> createProducer(taskId)
                                     );
                                 }
                         
                        @@ -486,7 +487,7 @@ StandbyTask createTask(final Consumer consumer,
                         
                                     final ProcessorTopology topology = builder.build(taskId.topicGroupId);
                         
                        -            if (!topology.stateStores().isEmpty()) {
                        +            if (!topology.stateStores().isEmpty() && !topology.storeToChangelogTopic().isEmpty()) {
                                         return new StandbyTask(
                                             taskId,
                                             partitions,
                        @@ -555,7 +556,7 @@ static class StreamsMetricsThreadImpl extends StreamsMetricsImpl {
                             }
                         
                             private final Time time;
                        -    private final long pollTimeMs;
                        +    private final Duration pollTime;
                             private final long commitTimeMs;
                             private final Object stateLock;
                             private final Logger log;
                        @@ -602,7 +603,8 @@ public static StreamThread create(final InternalTopologyBuilder builder,
                                 log.info("Creating restore consumer client");
                                 final Map restoreConsumerConfigs = config.getRestoreConsumerConfigs(threadClientId);
                                 final Consumer restoreConsumer = clientSupplier.getRestoreConsumer(restoreConsumerConfigs);
                        -        final StoreChangelogReader changelogReader = new StoreChangelogReader(restoreConsumer, userStateRestoreListener, logContext);
                        +        final Duration pollTime = Duration.ofMillis(config.getLong(StreamsConfig.POLL_MS_CONFIG));
                        +        final StoreChangelogReader changelogReader = new StoreChangelogReader(restoreConsumer, pollTime, userStateRestoreListener, logContext);
                         
                                 Producer threadProducer = null;
                                 final boolean eosEnabled = StreamsConfig.EXACTLY_ONCE.equals(config.getString(StreamsConfig.PROCESSING_GUARANTEE_CONFIG));
                        @@ -710,10 +712,10 @@ public StreamThread(final Time time,
                                 this.originalReset = originalReset;
                                 this.versionProbingFlag = versionProbingFlag;
                         
                        -        this.pollTimeMs = config.getLong(StreamsConfig.POLL_MS_CONFIG);
                        +        this.pollTime = Duration.ofMillis(config.getLong(StreamsConfig.POLL_MS_CONFIG));
                                 this.commitTimeMs = config.getLong(StreamsConfig.COMMIT_INTERVAL_MS_CONFIG);
                         
                        -        updateThreadMetadata(Collections.emptyMap(), Collections.emptyMap());
                        +        updateThreadMetadata(Collections.emptyMap(), Collections.emptyMap());
                             }
                         
                             /**
                        @@ -801,22 +803,27 @@ long runOnce(final long recordsProcessedBeforeCommit) {
                                 if (state == State.PARTITIONS_ASSIGNED) {
                                     // try to fetch some records with zero poll millis
                                     // to unblock the restoration as soon as possible
                        -            records = pollRequests(0L);
                        +            records = pollRequests(Duration.ZERO);
                        +        } else if (state == State.PARTITIONS_REVOKED) {
                        +            // try to fetch some records with normal poll time
                        +            // in order to wait long enough to get the join response
                        +            records = pollRequests(pollTime);
                        +        } else if (state == State.RUNNING) {
                        +            // try to fetch some records with normal poll time
                        +            // in order to get long polling
                        +            records = pollRequests(pollTime);
                        +        } else {
                        +            // any other state should not happen
                        +            log.error("Unexpected state {} during normal iteration", state);
                        +            throw new StreamsException(logPrefix + "Unexpected state " + state + " during normal iteration");
                        +        }
                         
                        +        // only try to initialize the assigned tasks
                        +        // if the state is still in PARTITION_ASSIGNED after the poll call
                        +        if (state == State.PARTITIONS_ASSIGNED) {
                                     if (taskManager.updateNewAndRestoringTasks()) {
                                         setState(State.RUNNING);
                                     }
                        -        } else {
                        -            // try to fetch some records if necessary
                        -            records = pollRequests(pollTimeMs);
                        -
                        -            // if state changed after the poll call,
                        -            // try to initialize the assigned tasks again
                        -            if (state == State.PARTITIONS_ASSIGNED) {
                        -                if (taskManager.updateNewAndRestoringTasks()) {
                        -                    setState(State.RUNNING);
                        -                }
                        -            }
                                 }
                         
                                 if (records != null && !records.isEmpty() && taskManager.hasActiveRunningTasks()) {
                        @@ -843,15 +850,15 @@ long runOnce(final long recordsProcessedBeforeCommit) {
                             /**
                              * Get the next batch of records by polling.
                              *
                        -     * @param pollTimeMs poll time millis parameter for the consumer poll
                        +     * @param pollTime how long to block in Consumer#poll
                              * @return Next batch of records or null if no records available.
                              * @throws TaskMigratedException if the task producer got fenced (EOS only)
                              */
                        -    private ConsumerRecords pollRequests(final long pollTimeMs) {
                        +    private ConsumerRecords pollRequests(final Duration pollTime) {
                                 ConsumerRecords records = null;
                         
                                 try {
                        -            records = consumer.poll(pollTimeMs);
                        +            records = consumer.poll(pollTime);
                                 } catch (final InvalidOffsetException e) {
                                     resetInvalidOffsets(e);
                                 }
                        @@ -920,9 +927,16 @@ private void addRecordsToTasks(final ConsumerRecords records) {
                                 for (final TopicPartition partition : records.partitions()) {
                                     final StreamTask task = taskManager.activeTask(partition);
                         
                        -            if (task.isClosed()) {
                        +            if (task == null) {
                        +                log.error(
                        +                    "Unable to locate active task for received-record partition {}. Current tasks: {}",
                        +                    partition,
                        +                    taskManager.toString(">")
                        +                );
                        +                throw new NullPointerException("Task was unexpectedly missing for partition " + partition);
                        +            } else if (task.isClosed()) {
                                         log.info("Stream task {} is already closed, probably because it got unexpectedly migrated to another thread already. " +
                        -                    "Notifying the thread to trigger a new rebalance immediately.", task.id());
                        +                             "Notifying the thread to trigger a new rebalance immediately.", task.id());
                                         throw new TaskMigratedException(task);
                                     }
                         
                        @@ -1078,7 +1092,11 @@ private void maybeUpdateStandbyTasks(final long now) {
                                     }
                         
                                     try {
                        -                final ConsumerRecords records = restoreConsumer.poll(0);
                        +                // poll(0): Since this is during the normal processing, not during restoration.
                        +                // We can afford to have slower restore (because we don't wait inside poll for results).
                        +                // Instead, we want to proceed to the next iteration to call the main consumer#poll()
                        +                // as soon as possible so as to not be kicked out of the group.
                        +                final ConsumerRecords records = restoreConsumer.poll(Duration.ZERO);
                         
                                         if (!records.isEmpty()) {
                                             for (final TopicPartition partition : records.partitions()) {
                        @@ -1113,7 +1131,7 @@ private void maybeUpdateStandbyTasks(final long now) {
                                                 throw new TaskMigratedException(task);
                                             }
                         
                        -                    log.info("Reinitializing StandbyTask {}", task);
                        +                    log.info("Reinitializing StandbyTask {} from changelogs {}", task, recoverableException.partitions());
                                             task.reinitializeStateStoresForPartitions(recoverableException.partitions());
                                         }
                                         restoreConsumer.seekToBeginning(partitions);
                        diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/Task.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/Task.java
                        index 5f221e3dc02e1..2b43640f00e31 100644
                        --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/Task.java
                        +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/Task.java
                        @@ -27,7 +27,7 @@
                         
                         public interface Task {
                             /**
                        -     * Initialize the task and return {}true if the task is ready to run, i.e, it has not state stores
                        +     * Initialize the task and return {@code true} if the task is ready to run, i.e, it has not state stores
                              * @return true if this task has no state stores that may need restoring.
                              * @throws IllegalStateException If store gets registered after initialized is already finished
                              * @throws StreamsException if the store's change log does not contain the partition
                        diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/assignment/StickyTaskAssignor.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/assignment/StickyTaskAssignor.java
                        index 5b54d08c03235..8767d0f6beac2 100644
                        --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/assignment/StickyTaskAssignor.java
                        +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/assignment/StickyTaskAssignor.java
                        @@ -270,7 +270,7 @@ boolean hasNewPair(final TaskId task1,
                                         if (!active && !pairs.contains(pair(task1, taskId))) {
                                             return true;
                                         }
                        -                if (!pairs.contains(pair(task1, taskId)) && task1.topicGroupId != taskId.topicGroupId) {
                        +                if (!pairs.contains(pair(task1, taskId))) {
                                             return true;
                                         }
                                     }
                        diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/metrics/StreamsMetricsImpl.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/metrics/StreamsMetricsImpl.java
                        index 662ded553addf..ba0b58fe2f002 100644
                        --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/metrics/StreamsMetricsImpl.java
                        +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/metrics/StreamsMetricsImpl.java
                        @@ -88,13 +88,13 @@ public final void removeAllThreadLevelSensors() {
                             }
                         
                             public final Sensor taskLevelSensor(final String taskName,
                        -                                         final String sensorName,
                        -                                         final Sensor.RecordingLevel recordingLevel,
                        -                                         final Sensor... parents) {
                        +                                        final String sensorName,
                        +                                        final Sensor.RecordingLevel recordingLevel,
                        +                                        final Sensor... parents) {
                                 final String key = threadName + "." + taskName;
                                 synchronized (taskLevelSensors) {
                                     if (!taskLevelSensors.containsKey(key)) {
                        -                taskLevelSensors.put(key, new LinkedList());
                        +                taskLevelSensors.put(key, new LinkedList<>());
                                     }
                         
                                     final String fullSensorName = key + "." + sensorName;
                        @@ -110,11 +110,9 @@ public final Sensor taskLevelSensor(final String taskName,
                             public final void removeAllTaskLevelSensors(final String taskName) {
                                 final String key = threadName + "." + taskName;
                                 synchronized (taskLevelSensors) {
                        -            if (taskLevelSensors.containsKey(key)) {
                        -                while (!taskLevelSensors.get(key).isEmpty()) {
                        -                    metrics.removeSensor(taskLevelSensors.get(key).pop());
                        -                }
                        -                taskLevelSensors.remove(key);
                        +            final Deque sensors = taskLevelSensors.remove(key);
                        +            while (sensors != null && !sensors.isEmpty()) {
                        +                metrics.removeSensor(sensors.pop());
                                     }
                                 }
                             }
                        @@ -127,7 +125,7 @@ public final Sensor cacheLevelSensor(final String taskName,
                                 final String key = threadName + "." + taskName + "." + cacheName;
                                 synchronized (cacheLevelSensors) {
                                     if (!cacheLevelSensors.containsKey(key)) {
                        -                cacheLevelSensors.put(key, new LinkedList());
                        +                cacheLevelSensors.put(key, new LinkedList<>());
                                     }
                         
                                     final String fullSensorName = key + "." + sensorName;
                        @@ -143,11 +141,9 @@ public final Sensor cacheLevelSensor(final String taskName,
                             public final void removeAllCacheLevelSensors(final String taskName, final String cacheName) {
                                 final String key = threadName + "." + taskName + "." + cacheName;
                                 synchronized (cacheLevelSensors) {
                        -            if (cacheLevelSensors.containsKey(key)) {
                        -                while (!cacheLevelSensors.get(key).isEmpty()) {
                        -                    metrics.removeSensor(cacheLevelSensors.get(key).pop());
                        -                }
                        -                cacheLevelSensors.remove(key);
                        +            final Deque strings = cacheLevelSensors.remove(key);
                        +            while (strings != null && !strings.isEmpty()) {
                        +                metrics.removeSensor(strings.pop());
                                     }
                                 }
                             }
                        @@ -361,10 +357,16 @@ public void removeSensor(final Sensor sensor) {
                                 Objects.requireNonNull(sensor, "Sensor is null");
                                 metrics.removeSensor(sensor.name());
                         
                        -        final Sensor parent = parentSensors.get(sensor);
                        +        final Sensor parent = parentSensors.remove(sensor);
                                 if (parent != null) {
                                     metrics.removeSensor(parent.name());
                                 }
                             }
                         
                        +    /**
                        +     * Visible for testing
                        +     */
                        +    Map parentSensors() {
                        +        return Collections.unmodifiableMap(parentSensors);
                        +    }
                         }
                        diff --git a/streams/src/main/java/org/apache/kafka/streams/state/StateSerdes.java b/streams/src/main/java/org/apache/kafka/streams/state/StateSerdes.java
                        index f1de82fd1992c..ec7803a7a0758 100644
                        --- a/streams/src/main/java/org/apache/kafka/streams/state/StateSerdes.java
                        +++ b/streams/src/main/java/org/apache/kafka/streams/state/StateSerdes.java
                        @@ -20,6 +20,7 @@
                         import org.apache.kafka.common.serialization.Serde;
                         import org.apache.kafka.common.serialization.Serdes;
                         import org.apache.kafka.common.serialization.Serializer;
                        +import org.apache.kafka.streams.errors.StreamsException;
                         
                         import java.util.Objects;
                         
                        @@ -165,7 +166,18 @@ public V valueFrom(byte[] rawValue) {
                              * @return     the serialized key
                              */
                             public byte[] rawKey(K key) {
                        -        return keySerde.serializer().serialize(topic, key);
                        +        try {
                        +            return keySerde.serializer().serialize(topic, key);
                        +        } catch (final ClassCastException e) {
                        +            final String keyClass = key == null ? "unknown because key is null" : key.getClass().getName();
                        +            throw new StreamsException(
                        +                    String.format("A serializer (key: %s) is not compatible to the actual key type " +
                        +                                    "(key type: %s). Change the default Serdes in StreamConfig or " +
                        +                                    "provide correct Serdes via method parameters.",
                        +                            keySerializer().getClass().getName(),
                        +                            keyClass),
                        +                    e);
                        +        }
                             }
                         
                             /**
                        @@ -175,6 +187,17 @@ public byte[] rawKey(K key) {
                              * @return       the serialized value
                              */
                             public byte[] rawValue(V value) {
                        -        return valueSerde.serializer().serialize(topic, value);
                        +        try {
                        +            return valueSerde.serializer().serialize(topic, value);
                        +        } catch (final ClassCastException e) {
                        +            final String valueClass = value == null ? "unknown because value is null" : value.getClass().getName();
                        +            throw new StreamsException(
                        +                    String.format("A serializer (value: %s) is not compatible to the actual value type " +
                        +                                    "(value type: %s). Change the default Serdes in StreamConfig or " +
                        +                                    "provide correct Serdes via method parameters.",
                        +                            valueSerializer().getClass().getName(),
                        +                            valueClass),
                        +                    e);
                        +        }
                             }
                         }
                        diff --git a/streams/src/main/java/org/apache/kafka/streams/state/internals/CachingKeyValueStore.java b/streams/src/main/java/org/apache/kafka/streams/state/internals/CachingKeyValueStore.java
                        index da308a1f5c794..5f1a938451eb7 100644
                        --- a/streams/src/main/java/org/apache/kafka/streams/state/internals/CachingKeyValueStore.java
                        +++ b/streams/src/main/java/org/apache/kafka/streams/state/internals/CachingKeyValueStore.java
                        @@ -129,9 +129,15 @@ public void flush() {
                         
                             @Override
                             public void close() {
                        -        flush();
                        -        underlying.close();
                        -        cache.close(cacheName);
                        +        try {
                        +            flush();
                        +        } finally {
                        +            try {
                        +                underlying.close();
                        +            } finally {
                        +                cache.close(cacheName);
                        +            }
                        +        }
                             }
                         
                             @Override
                        diff --git a/streams/src/main/java/org/apache/kafka/streams/state/internals/CachingSessionStore.java b/streams/src/main/java/org/apache/kafka/streams/state/internals/CachingSessionStore.java
                        index 69506932e5350..c307f6d762865 100644
                        --- a/streams/src/main/java/org/apache/kafka/streams/state/internals/CachingSessionStore.java
                        +++ b/streams/src/main/java/org/apache/kafka/streams/state/internals/CachingSessionStore.java
                        @@ -147,7 +147,7 @@ public void put(final Windowed key, byte[] value) {
                                         context.headers(),
                                         true,
                                         context.offset(),
                        -                key.window().end(),
                        +                context.timestamp(),
                                         context.partition(),
                                         context.topic());
                                 cache.put(cacheName, cacheFunction.cacheKey(binaryKey), entry);
                        diff --git a/streams/src/main/java/org/apache/kafka/streams/state/internals/CachingWindowStore.java b/streams/src/main/java/org/apache/kafka/streams/state/internals/CachingWindowStore.java
                        index 1f08f5157b94b..07120dffbbffb 100644
                        --- a/streams/src/main/java/org/apache/kafka/streams/state/internals/CachingWindowStore.java
                        +++ b/streams/src/main/java/org/apache/kafka/streams/state/internals/CachingWindowStore.java
                        @@ -156,7 +156,7 @@ public synchronized void put(final Bytes key, final byte[] value, final long tim
                                         context.headers(),
                                         true,
                                         context.offset(),
                        -                timestamp,
                        +                context.timestamp(),
                                         context.partition(),
                                         context.topic());
                                 cache.put(name, cacheFunction.cacheKey(keyBytes), entry);
                        diff --git a/streams/src/main/java/org/apache/kafka/streams/state/internals/InnerMeteredKeyValueStore.java b/streams/src/main/java/org/apache/kafka/streams/state/internals/InnerMeteredKeyValueStore.java
                        index 14464e09b0002..200b2d7870a24 100644
                        --- a/streams/src/main/java/org/apache/kafka/streams/state/internals/InnerMeteredKeyValueStore.java
                        +++ b/streams/src/main/java/org/apache/kafka/streams/state/internals/InnerMeteredKeyValueStore.java
                        @@ -19,6 +19,7 @@
                         import org.apache.kafka.common.metrics.Sensor;
                         import org.apache.kafka.common.utils.Time;
                         import org.apache.kafka.streams.KeyValue;
                        +import org.apache.kafka.streams.errors.ProcessorStateException;
                         import org.apache.kafka.streams.processor.ProcessorContext;
                         import org.apache.kafka.streams.processor.StateStore;
                         import org.apache.kafka.streams.processor.internals.metrics.StreamsMetricsImpl;
                        @@ -173,30 +174,40 @@ private interface Action {
                         
                             @Override
                             public V get(final K key) {
                        -        if (getTime.shouldRecord()) {
                        -            return measureLatency(new Action() {
                        -                @Override
                        -                public V execute() {
                        -                    return typeConverter.outerValue(inner.get(typeConverter.innerKey(key)));
                        -                }
                        -            }, getTime);
                        -        } else {
                        -            return typeConverter.outerValue(inner.get(typeConverter.innerKey(key)));
                        +        try {
                        +            if (getTime.shouldRecord()) {
                        +                return measureLatency(new Action() {
                        +                    @Override
                        +                    public V execute() {
                        +                        return typeConverter.outerValue(inner.get(typeConverter.innerKey(key)));
                        +                    }
                        +                }, getTime);
                        +            } else {
                        +                return typeConverter.outerValue(inner.get(typeConverter.innerKey(key)));
                        +            }
                        +        } catch (final ProcessorStateException e) {
                        +            final String message = String.format(e.getMessage(), key);
                        +            throw new ProcessorStateException(message, e);
                                 }
                             }
                         
                             @Override
                             public void put(final K key, final V value) {
                        -        if (putTime.shouldRecord()) {
                        -            measureLatency(new Action() {
                        -                @Override
                        -                public V execute() {
                        -                    inner.put(typeConverter.innerKey(key), typeConverter.innerValue(value));
                        -                    return null;
                        -                }
                        -            }, putTime);
                        -        } else {
                        -            inner.put(typeConverter.innerKey(key), typeConverter.innerValue(value));
                        +        try {
                        +            if (putTime.shouldRecord()) {
                        +                measureLatency(new Action() {
                        +                    @Override
                        +                    public V execute() {
                        +                        inner.put(typeConverter.innerKey(key), typeConverter.innerValue(value));
                        +                        return null;
                        +                    }
                        +                }, putTime);
                        +            } else {
                        +                inner.put(typeConverter.innerKey(key), typeConverter.innerValue(value));
                        +            }
                        +        } catch (final ProcessorStateException e) {
                        +            final String message = String.format(e.getMessage(), key, value);
                        +            throw new ProcessorStateException(message, e);
                                 }
                             }
                         
                        @@ -232,15 +243,20 @@ public V execute() {
                         
                             @Override
                             public V delete(final K key) {
                        -        if (deleteTime.shouldRecord()) {
                        -            return measureLatency(new Action() {
                        -                @Override
                        -                public V execute() {
                        -                    return typeConverter.outerValue(inner.delete(typeConverter.innerKey(key)));
                        -                }
                        -            }, deleteTime);
                        -        } else {
                        -            return typeConverter.outerValue(inner.delete(typeConverter.innerKey(key)));
                        +        try {
                        +            if (deleteTime.shouldRecord()) {
                        +                return measureLatency(new Action() {
                        +                    @Override
                        +                    public V execute() {
                        +                        return typeConverter.outerValue(inner.delete(typeConverter.innerKey(key)));
                        +                    }
                        +                }, deleteTime);
                        +            } else {
                        +                return typeConverter.outerValue(inner.delete(typeConverter.innerKey(key)));
                        +            }
                        +        } catch (final ProcessorStateException e) {
                        +            final String message = String.format(e.getMessage(), key);
                        +            throw new ProcessorStateException(message, e);
                                 }
                             }
                         
                        diff --git a/streams/src/main/java/org/apache/kafka/streams/state/internals/MeteredSessionStore.java b/streams/src/main/java/org/apache/kafka/streams/state/internals/MeteredSessionStore.java
                        index 563621971f5bf..3e881edc5c6f2 100644
                        --- a/streams/src/main/java/org/apache/kafka/streams/state/internals/MeteredSessionStore.java
                        +++ b/streams/src/main/java/org/apache/kafka/streams/state/internals/MeteredSessionStore.java
                        @@ -20,6 +20,7 @@
                         import org.apache.kafka.common.serialization.Serde;
                         import org.apache.kafka.common.utils.Bytes;
                         import org.apache.kafka.common.utils.Time;
                        +import org.apache.kafka.streams.errors.ProcessorStateException;
                         import org.apache.kafka.streams.kstream.Windowed;
                         import org.apache.kafka.streams.processor.ProcessorContext;
                         import org.apache.kafka.streams.processor.StateStore;
                        @@ -128,6 +129,9 @@ public void remove(final Windowed sessionKey) {
                                 try {
                                     final Bytes key = keyBytes(sessionKey.key());
                                     inner.remove(new Windowed<>(key, sessionKey.window()));
                        +        } catch (final ProcessorStateException e) {
                        +            final String message = String.format(e.getMessage(), sessionKey.key());
                        +            throw new ProcessorStateException(message, e);
                                 } finally {
                                     this.metrics.recordLatency(removeTime, startNs, time.nanoseconds());
                                 }
                        @@ -140,6 +144,9 @@ public void put(final Windowed sessionKey, final V aggregate) {
                                 try {
                                     final Bytes key = keyBytes(sessionKey.key());
                                     this.inner.put(new Windowed<>(key, sessionKey.window()), serdes.rawValue(aggregate));
                        +        } catch (final ProcessorStateException e) {
                        +            final String message = String.format(e.getMessage(), sessionKey.key(), aggregate);
                        +            throw new ProcessorStateException(message, e);
                                 } finally {
                                     this.metrics.recordLatency(this.putTime, startNs, time.nanoseconds());
                                 }
                        diff --git a/streams/src/main/java/org/apache/kafka/streams/state/internals/MeteredWindowStore.java b/streams/src/main/java/org/apache/kafka/streams/state/internals/MeteredWindowStore.java
                        index 24878543a7e10..62ed6c6ce0c38 100644
                        --- a/streams/src/main/java/org/apache/kafka/streams/state/internals/MeteredWindowStore.java
                        +++ b/streams/src/main/java/org/apache/kafka/streams/state/internals/MeteredWindowStore.java
                        @@ -20,6 +20,7 @@
                         import org.apache.kafka.common.serialization.Serde;
                         import org.apache.kafka.common.utils.Bytes;
                         import org.apache.kafka.common.utils.Time;
                        +import org.apache.kafka.streams.errors.ProcessorStateException;
                         import org.apache.kafka.streams.kstream.Windowed;
                         import org.apache.kafka.streams.processor.ProcessorContext;
                         import org.apache.kafka.streams.processor.StateStore;
                        @@ -94,6 +95,9 @@ public void put(final K key, final V value, final long timestamp) {
                                 final long startNs = time.nanoseconds();
                                 try {
                                     inner.put(keyBytes(key), serdes.rawValue(value), timestamp);
                        +        } catch (final ProcessorStateException e) {
                        +            final String message = String.format(e.getMessage(), key, value);
                        +            throw new ProcessorStateException(message, e);
                                 } finally {
                                     metrics.recordLatency(this.putTime, startNs, time.nanoseconds());
                                 }
                        diff --git a/streams/src/main/java/org/apache/kafka/streams/state/internals/NamedCache.java b/streams/src/main/java/org/apache/kafka/streams/state/internals/NamedCache.java
                        index 77b9c1e856074..8f4166e9bce33 100644
                        --- a/streams/src/main/java/org/apache/kafka/streams/state/internals/NamedCache.java
                        +++ b/streams/src/main/java/org/apache/kafka/streams/state/internals/NamedCache.java
                        @@ -270,6 +270,10 @@ synchronized Iterator keyRange(final Bytes from, final Bytes to) {
                                 return keySetIterator(cache.navigableKeySet().subSet(from, true, to, true));
                             }
                         
                        +    public boolean isEmpty() {
                        +        return cache.isEmpty();
                        +    }
                        +
                             private Iterator keySetIterator(final Set keySet) {
                                 return new TreeSet<>(keySet).iterator();
                             }
                        @@ -370,7 +374,7 @@ private NamedCacheMetrics(final StreamsMetricsImpl metrics, final String cacheNa
                                         "record-cache-id", "all",
                                         "task-id", taskName
                                     );
                        -            final Sensor taskLevelHitRatioSensor = metrics.taskLevelSensor("hitRatio", taskName, Sensor.RecordingLevel.DEBUG);
                        +            final Sensor taskLevelHitRatioSensor = metrics.taskLevelSensor(taskName, "hitRatio", Sensor.RecordingLevel.DEBUG);
                                     taskLevelHitRatioSensor.add(
                                         new MetricName("hitRatio-avg", group, "The average cache hit ratio.", allMetricTags),
                                         new Avg()
                        diff --git a/streams/src/main/java/org/apache/kafka/streams/state/internals/RocksDBStore.java b/streams/src/main/java/org/apache/kafka/streams/state/internals/RocksDBStore.java
                        index ff6c56add84d0..6084ecbf1e0ea 100644
                        --- a/streams/src/main/java/org/apache/kafka/streams/state/internals/RocksDBStore.java
                        +++ b/streams/src/main/java/org/apache/kafka/streams/state/internals/RocksDBStore.java
                        @@ -130,10 +130,6 @@ public void openDB(final ProcessorContext context) {
                                 // (this could be a bug in the RocksDB code and their devs have been contacted).
                                 options.setIncreaseParallelism(Math.max(Runtime.getRuntime().availableProcessors(), 2));
                         
                        -        if (prepareForBulkload) {
                        -            options.prepareForBulkLoad();
                        -        }
                        -
                                 wOptions = new WriteOptions();
                                 wOptions.setDisableWAL(true);
                         
                        @@ -148,6 +144,11 @@ public void openDB(final ProcessorContext context) {
                                     final RocksDBConfigSetter configSetter = Utils.newInstance(configSetterClass);
                                     configSetter.setConfig(name, options, configs);
                                 }
                        +
                        +        if (prepareForBulkload) {
                        +            options.prepareForBulkLoad();
                        +        }
                        +
                                 this.dbDir = new File(new File(context.stateDir(), parentDir), this.name);
                         
                                 try {
                        @@ -224,7 +225,8 @@ private byte[] getInternal(final byte[] rawKey) {
                                 try {
                                     return this.db.get(rawKey);
                                 } catch (final RocksDBException e) {
                        -            throw new ProcessorStateException("Error while getting value for key from store " + this.name, e);
                        +            // String format is happening in wrapping stores. So formatted message is thrown from wrapping stores.
                        +            throw new ProcessorStateException("Error while getting value for key %s from store " + this.name, e);
                                 }
                             }
                         
                        @@ -300,13 +302,15 @@ private void putInternal(final byte[] rawKey,
                                     try {
                                         db.delete(wOptions, rawKey);
                                     } catch (final RocksDBException e) {
                        -                throw new ProcessorStateException("Error while removing key from store " + this.name, e);
                        +                // String format is happening in wrapping stores. So formatted message is thrown from wrapping stores.
                        +                throw new ProcessorStateException("Error while removing key %s from store " + this.name, e);
                                     }
                                 } else {
                                     try {
                                         db.put(wOptions, rawKey, rawValue);
                                     } catch (final RocksDBException e) {
                        -                throw new ProcessorStateException("Error while executing putting key/value into store " + this.name, e);
                        +                // String format is happening in wrapping stores. So formatted message is thrown from wrapping stores.
                        +                throw new ProcessorStateException("Error while putting key %s value %s into store " + this.name, e);
                                     }
                                 }
                             }
                        diff --git a/streams/src/main/java/org/apache/kafka/streams/state/internals/RocksDbSessionBytesStoreSupplier.java b/streams/src/main/java/org/apache/kafka/streams/state/internals/RocksDbSessionBytesStoreSupplier.java
                        index 5a87bc57d56d5..df7a8be6d69fc 100644
                        --- a/streams/src/main/java/org/apache/kafka/streams/state/internals/RocksDbSessionBytesStoreSupplier.java
                        +++ b/streams/src/main/java/org/apache/kafka/streams/state/internals/RocksDbSessionBytesStoreSupplier.java
                        @@ -50,7 +50,7 @@ public SessionStore get() {
                         
                             @Override
                             public String metricsScope() {
                        -        return "rocksdb-session";
                        +        return "rocksdb-session-state";
                             }
                         
                             @Override
                        diff --git a/streams/src/main/java/org/apache/kafka/streams/state/internals/RocksDbWindowBytesStoreSupplier.java b/streams/src/main/java/org/apache/kafka/streams/state/internals/RocksDbWindowBytesStoreSupplier.java
                        index 5fbf491dfc414..73c3630855d44 100644
                        --- a/streams/src/main/java/org/apache/kafka/streams/state/internals/RocksDbWindowBytesStoreSupplier.java
                        +++ b/streams/src/main/java/org/apache/kafka/streams/state/internals/RocksDbWindowBytesStoreSupplier.java
                        @@ -68,7 +68,7 @@ public WindowStore get() {
                         
                             @Override
                             public String metricsScope() {
                        -        return "rocksdb-window";
                        +        return "rocksdb-window-state";
                             }
                         
                             @Override
                        diff --git a/streams/src/main/java/org/apache/kafka/streams/state/internals/ThreadCache.java b/streams/src/main/java/org/apache/kafka/streams/state/internals/ThreadCache.java
                        index 7ce03a18c0162..ef8020e6b559a 100644
                        --- a/streams/src/main/java/org/apache/kafka/streams/state/internals/ThreadCache.java
                        +++ b/streams/src/main/java/org/apache/kafka/streams/state/internals/ThreadCache.java
                        @@ -193,7 +193,7 @@ public MemoryLRUCacheBytesIterator all(final String namespace) {
                                 }
                                 return new MemoryLRUCacheBytesIterator(cache.allKeys(), cache);
                             }
                        -    
                        +
                             public long size() {
                                 long size = 0;
                                 for (NamedCache cache : caches.values()) {
                        @@ -235,7 +235,7 @@ private void maybeEvict(final String namespace) {
                                     // a put on another cache. So even though the sizeInBytes() is
                                     // still > maxCacheSizeBytes there is nothing to evict from this
                                     // namespaced cache.
                        -            if (cache.size() == 0) {
                        +            if (cache.isEmpty()) {
                                         return;
                                     }
                                     cache.evict();
                        diff --git a/streams/src/main/java/org/apache/kafka/streams/state/internals/WindowStoreBuilder.java b/streams/src/main/java/org/apache/kafka/streams/state/internals/WindowStoreBuilder.java
                        index 97b4883084c1d..cd0841a0bf797 100644
                        --- a/streams/src/main/java/org/apache/kafka/streams/state/internals/WindowStoreBuilder.java
                        +++ b/streams/src/main/java/org/apache/kafka/streams/state/internals/WindowStoreBuilder.java
                        @@ -52,7 +52,7 @@ private WindowStore maybeWrapCaching(final WindowStore maybeWrapLogging(final WindowStore inner) {
                        diff --git a/streams/src/test/java/org/apache/kafka/streams/KafkaStreamsTest.java b/streams/src/test/java/org/apache/kafka/streams/KafkaStreamsTest.java
                        index 297b2434c06cb..0ac1ebf367f24 100644
                        --- a/streams/src/test/java/org/apache/kafka/streams/KafkaStreamsTest.java
                        +++ b/streams/src/test/java/org/apache/kafka/streams/KafkaStreamsTest.java
                        @@ -16,6 +16,7 @@
                          */
                         package org.apache.kafka.streams;
                         
                        +import org.apache.kafka.clients.consumer.ConsumerConfig;
                         import org.apache.kafka.clients.producer.MockProducer;
                         import org.apache.kafka.common.Cluster;
                         import org.apache.kafka.common.Node;
                        @@ -25,6 +26,7 @@
                         import org.apache.kafka.common.serialization.Serdes;
                         import org.apache.kafka.common.serialization.StringSerializer;
                         import org.apache.kafka.common.utils.Utils;
                        +import org.apache.kafka.streams.errors.StreamsException;
                         import org.apache.kafka.streams.integration.utils.EmbeddedKafkaCluster;
                         import org.apache.kafka.streams.integration.utils.IntegrationTestUtils;
                         import org.apache.kafka.streams.kstream.Consumed;
                        @@ -42,7 +44,6 @@
                         import org.junit.Assert;
                         import org.junit.Before;
                         import org.junit.ClassRule;
                        -import org.junit.Ignore;
                         import org.junit.Test;
                         import org.junit.experimental.categories.Category;
                         
                        @@ -234,9 +235,8 @@ public boolean conditionMet() {
                                 assertEquals(streams.state(), KafkaStreams.State.NOT_RUNNING);
                             }
                         
                        -    @Ignore // this test cannot pass as long as GST blocks KS.start()
                             @Test
                        -    public void testGlobalThreadCloseWithoutConnectingToBroker() {
                        +    public void globalThreadShouldTimeoutWhenBrokerConnectionCannotBeEstablished() {
                                 final Properties props = new Properties();
                                 props.setProperty(StreamsConfig.APPLICATION_ID_CONFIG, "appId");
                                 props.setProperty(StreamsConfig.BOOTSTRAP_SERVERS_CONFIG, "localhost:1");
                        @@ -244,16 +244,21 @@ public void testGlobalThreadCloseWithoutConnectingToBroker() {
                                 props.setProperty(StreamsConfig.STATE_DIR_CONFIG, TestUtils.tempDirectory().getPath());
                                 props.put(StreamsConfig.NUM_STREAM_THREADS_CONFIG, NUM_THREADS);
                         
                        +        props.put(ConsumerConfig.DEFAULT_API_TIMEOUT_MS_CONFIG, 200);
                        +
                                 final StreamsBuilder builder = new StreamsBuilder();
                                 // make sure we have the global state thread running too
                                 builder.globalTable("anyTopic");
                                 final KafkaStreams streams = new KafkaStreams(builder.build(), props);
                        -        streams.start();
                        -        streams.close();
                        +        try {
                        +            streams.start();
                        +            fail("expected start() to time out and throw an exception.");
                        +        } catch (final StreamsException expected) {
                        +            // This is a result of not being able to connect to the broker.
                        +        }
                                 // There's nothing to assert... We're testing that this operation actually completes.
                             }
                         
                        -    @Ignore // this test cannot pass until we implement KIP-266
                             @Test
                             public void testLocalThreadCloseWithoutConnectingToBroker() {
                                 final Properties props = new Properties();
                        @@ -451,6 +456,9 @@ public void apply(final String key, final String value) {
                                 } finally {
                                     // stop the thread so we don't interfere with other tests etc
                                     keepRunning.set(false);
                        +            if (streams != null) {
                        +                streams.close();
                        +            }
                                 }
                             }
                         
                        diff --git a/streams/src/test/java/org/apache/kafka/streams/StreamsBuilderTest.java b/streams/src/test/java/org/apache/kafka/streams/StreamsBuilderTest.java
                        index 37101de344a50..3b8c9bd47d9c5 100644
                        --- a/streams/src/test/java/org/apache/kafka/streams/StreamsBuilderTest.java
                        +++ b/streams/src/test/java/org/apache/kafka/streams/StreamsBuilderTest.java
                        @@ -273,11 +273,17 @@ public void shouldUseDefaultNodeAndStoreNames() {
                             }
                         
                             @Test
                        -    public void shouldReuseSourceTopicAsChangelogs() {
                        +    public void shouldReuseSourceTopicAsChangelogsWithOptimization20() {
                                 final String topic = "topic";
                                 builder.table(topic, Materialized.>as("store"));
                        +        final Topology topology = builder.build();
                        +        final Properties props = StreamsTestUtils.minimalStreamsConfig();
                        +        props.put(StreamsConfig.TOPOLOGY_OPTIMIZATION, StreamsConfig.OPTIMIZE);
                         
                        -        final InternalTopologyBuilder internalTopologyBuilder = TopologyWrapper.getInternalTopologyBuilder(builder.build());
                        +        final InternalTopologyBuilder internalTopologyBuilder = TopologyWrapper.getInternalTopologyBuilder(topology);
                        +        internalTopologyBuilder.adjust(new StreamsConfig(props));
                        +
                        +        assertThat(internalTopologyBuilder.build().storeToChangelogTopic(), equalTo(Collections.singletonMap("store", "topic")));
                         
                                 assertThat(internalTopologyBuilder.getStateStores().keySet(), equalTo(Collections.singleton("store")));
                         
                        @@ -285,6 +291,23 @@ public void shouldReuseSourceTopicAsChangelogs() {
                         
                                 assertThat(internalTopologyBuilder.topicGroups().get(0).stateChangelogTopics.isEmpty(), equalTo(true));
                             }
                        +
                        +    @Test
                        +    public void shouldNotReuseSourceTopicAsChangelogsByDefault() {
                        +        final String topic = "topic";
                        +        builder.table(topic, Materialized.>as("store"));
                        +
                        +        final InternalTopologyBuilder internalTopologyBuilder = TopologyWrapper.getInternalTopologyBuilder(builder.build());
                        +        internalTopologyBuilder.setApplicationId("appId");
                        +
                        +        assertThat(internalTopologyBuilder.build().storeToChangelogTopic(), equalTo(Collections.singletonMap("store", "appId-store-changelog")));
                        +
                        +        assertThat(internalTopologyBuilder.getStateStores().keySet(), equalTo(Collections.singleton("store")));
                        +
                        +        assertThat(internalTopologyBuilder.getStateStores().get("store").loggingEnabled(), equalTo(true));
                        +
                        +        assertThat(internalTopologyBuilder.topicGroups().get(0).stateChangelogTopics.keySet(), equalTo(Collections.singleton("appId-store-changelog")));
                        +    }
                             
                             @Test(expected = TopologyException.class)
                             public void shouldThrowExceptionWhenNoTopicPresent() {
                        diff --git a/streams/src/test/java/org/apache/kafka/streams/StreamsConfigTest.java b/streams/src/test/java/org/apache/kafka/streams/StreamsConfigTest.java
                        index e991b6ff89e54..05995fed619c6 100644
                        --- a/streams/src/test/java/org/apache/kafka/streams/StreamsConfigTest.java
                        +++ b/streams/src/test/java/org/apache/kafka/streams/StreamsConfigTest.java
                        @@ -45,9 +45,11 @@
                         import static org.apache.kafka.common.requests.IsolationLevel.READ_COMMITTED;
                         import static org.apache.kafka.common.requests.IsolationLevel.READ_UNCOMMITTED;
                         import static org.apache.kafka.streams.StreamsConfig.EXACTLY_ONCE;
                        +import static org.apache.kafka.streams.StreamsConfig.TOPOLOGY_OPTIMIZATION;
                         import static org.apache.kafka.streams.StreamsConfig.adminClientPrefix;
                         import static org.apache.kafka.streams.StreamsConfig.consumerPrefix;
                         import static org.apache.kafka.streams.StreamsConfig.producerPrefix;
                        +import static org.apache.kafka.streams.StreamsConfig.topicPrefix;
                         import static org.apache.kafka.test.StreamsTestUtils.minimalStreamsConfig;
                         import static org.hamcrest.core.IsEqual.equalTo;
                         import static org.junit.Assert.assertEquals;
                        @@ -110,7 +112,7 @@ public void consumerConfigMustContainStreamPartitionAssignorConfig() {
                                 props.put(StreamsConfig.APPLICATION_SERVER_CONFIG, "dummy:host");
                                 props.put(StreamsConfig.RETRIES_CONFIG, 10);
                                 props.put(StreamsConfig.adminClientPrefix(StreamsConfig.RETRIES_CONFIG), 5);
                        -        props.put(StreamsConfig.topicPrefix(TopicConfig.SEGMENT_BYTES_CONFIG), 100);
                        +        props.put(topicPrefix(TopicConfig.SEGMENT_BYTES_CONFIG), 100);
                                 final StreamsConfig streamsConfig = new StreamsConfig(props);
                         
                                 final String groupId = "example-application";
                        @@ -122,15 +124,17 @@ public void consumerConfigMustContainStreamPartitionAssignorConfig() {
                                 assertEquals(StreamsPartitionAssignor.class.getName(), returnedProps.get(ConsumerConfig.PARTITION_ASSIGNMENT_STRATEGY_CONFIG));
                                 assertEquals(7L, returnedProps.get(StreamsConfig.WINDOW_STORE_CHANGE_LOG_ADDITIONAL_RETENTION_MS_CONFIG));
                                 assertEquals("dummy:host", returnedProps.get(StreamsConfig.APPLICATION_SERVER_CONFIG));
                        -        assertEquals(null, returnedProps.get(StreamsConfig.RETRIES_CONFIG));
                        +        assertNull(returnedProps.get(StreamsConfig.RETRIES_CONFIG));
                                 assertEquals(5, returnedProps.get(StreamsConfig.adminClientPrefix(StreamsConfig.RETRIES_CONFIG)));
                        -        assertEquals(100, returnedProps.get(StreamsConfig.topicPrefix(TopicConfig.SEGMENT_BYTES_CONFIG)));
                        +        assertEquals(100, returnedProps.get(topicPrefix(TopicConfig.SEGMENT_BYTES_CONFIG)));
                             }
                         
                             @Test
                        -    public void consumerConfigMustUseAdminClientConfigForRetries() {
                        +    public void consumerConfigShouldContainAdminClientConfigsForRetriesAndRetryBackOffMsWithAdminPrefix() {
                                 props.put(StreamsConfig.adminClientPrefix(StreamsConfig.RETRIES_CONFIG), 20);
                        +        props.put(StreamsConfig.adminClientPrefix(StreamsConfig.RETRY_BACKOFF_MS_CONFIG), 200L);
                                 props.put(StreamsConfig.RETRIES_CONFIG, 10);
                        +        props.put(StreamsConfig.RETRY_BACKOFF_MS_CONFIG, 100L);
                                 final StreamsConfig streamsConfig = new StreamsConfig(props);
                         
                                 final String groupId = "example-application";
                        @@ -138,6 +142,7 @@ public void consumerConfigMustUseAdminClientConfigForRetries() {
                                 final Map returnedProps = streamsConfig.getMainConsumerConfigs(groupId, clientId);
                         
                                 assertEquals(20, returnedProps.get(StreamsConfig.adminClientPrefix(StreamsConfig.RETRIES_CONFIG)));
                        +        assertEquals(200L, returnedProps.get(StreamsConfig.adminClientPrefix(StreamsConfig.RETRY_BACKOFF_MS_CONFIG)));
                             }
                         
                             @Test
                        @@ -232,7 +237,6 @@ public void shouldSupportPrefixedPropertiesThatAreNotPartOfProducerConfig() {
                                 assertEquals("host", producerConfigs.get("interceptor.statsd.host"));
                             }
                         
                        -
                             @Test
                             public void shouldSupportPrefixedProducerConfigs() {
                                 props.put(producerPrefix(ProducerConfig.BUFFER_MEMORY_CONFIG), 10);
                        @@ -426,7 +430,7 @@ public void testGetGlobalConsumerConfigsWithGlobalConsumerOverridenPrefix() {
                             public void shouldSetInternalLeaveGroupOnCloseConfigToFalseInConsumer() {
                                 final StreamsConfig streamsConfig = new StreamsConfig(props);
                                 final Map consumerConfigs = streamsConfig.getMainConsumerConfigs("groupId", "clientId");
                        -        assertThat(consumerConfigs.get("internal.leave.group.on.close"), CoreMatchers.equalTo(false));
                        +        assertThat(consumerConfigs.get("internal.leave.group.on.close"), CoreMatchers.equalTo(false));
                             }
                         
                             @Test
                        @@ -571,18 +575,67 @@ public void shouldSpecifyCorrectValueSerdeClassOnError() {
                             }
                         
                             @Test
                        -    public void shouldThrowExceptionIfMaxInflightRequestsGreatherThanFiveIfEosEnabled() {
                        -        props.put(ProducerConfig.MAX_IN_FLIGHT_REQUESTS_PER_CONNECTION, 7);
                        +    public void shouldThrowExceptionIfMaxInFlightRequestsGreaterThanFiveIfEosEnabled() {
                                 props.put(StreamsConfig.PROCESSING_GUARANTEE_CONFIG, EXACTLY_ONCE);
                        +        props.put(ProducerConfig.MAX_IN_FLIGHT_REQUESTS_PER_CONNECTION, 7);
                                 final StreamsConfig streamsConfig = new StreamsConfig(props);
                                 try {
                                     streamsConfig.getProducerConfigs("clientId");
                        -            fail("Should throw ConfigException when Eos is enabled and maxInFlight requests exceeds 5");
                        +            fail("Should throw ConfigException when ESO is enabled and maxInFlight requests exceeds 5");
                        +        } catch (final ConfigException e) {
                        +            assertEquals("Invalid value 7 for configuration max.in.flight.requests.per.connection: Can't exceed 5 when exactly-once processing is enabled", e.getMessage());
                        +        }
                        +    }
                        +
                        +    @Test
                        +    public void shouldAllowToSpecifyMaxInFlightRequestsPerConnectionAsStringIfEosEnabled() {
                        +        props.put(StreamsConfig.PROCESSING_GUARANTEE_CONFIG, EXACTLY_ONCE);
                        +        props.put(ProducerConfig.MAX_IN_FLIGHT_REQUESTS_PER_CONNECTION, "3");
                        +
                        +        new StreamsConfig(props).getProducerConfigs("clientId");
                        +    }
                        +
                        +    @Test
                        +    public void shouldThrowConfigExceptionIfMaxInFlightRequestsPerConnectionIsInvalidStringIfEosEnabled() {
                        +        props.put(StreamsConfig.PROCESSING_GUARANTEE_CONFIG, EXACTLY_ONCE);
                        +        props.put(ProducerConfig.MAX_IN_FLIGHT_REQUESTS_PER_CONNECTION, "not-a-number");
                        +
                        +        try {
                        +            new StreamsConfig(props).getProducerConfigs("clientId");
                        +            fail("Should throw ConfigException when EOS is enabled and maxInFlight cannot be paresed into an integer");
                                 } catch (final ConfigException e) {
                        -            assertEquals("max.in.flight.requests.per.connection can't exceed 5 when using the idempotent producer", e.getMessage());
                        +            assertEquals("Invalid value not-a-number for configuration max.in.flight.requests.per.connection: String value could not be parsed as 32-bit integer", e.getMessage());
                                 }
                             }
                         
                        +    @Test
                        +    public void shouldSpecifyNoOptimizationWhenNotExplicitlyAddedToConfigs() {
                        +        final String expectedOptimizeConfig = "none";
                        +        final String actualOptimizedConifig = streamsConfig.getString(TOPOLOGY_OPTIMIZATION);
                        +        assertEquals("Optimization should be \"none\"", expectedOptimizeConfig, actualOptimizedConifig);
                        +    }
                        +
                        +    @Test
                        +    public void shouldSpecifyOptimizationWhenNotExplicitlyAddedToConfigs() {
                        +        final String expectedOptimizeConfig = "all";
                        +        props.put(TOPOLOGY_OPTIMIZATION, "all");
                        +        final StreamsConfig config = new StreamsConfig(props);
                        +        final String actualOptimizedConifig = config.getString(TOPOLOGY_OPTIMIZATION);
                        +        assertEquals("Optimization should be \"all\"", expectedOptimizeConfig, actualOptimizedConifig);
                        +    }
                        +
                        +    @Test(expected = ConfigException.class)
                        +    public void shouldThrowConfigExceptionWhenOptimizationConfigNotValueInRange() {
                        +        props.put(TOPOLOGY_OPTIMIZATION, "maybe");
                        +        new StreamsConfig(props);
                        +    }
                        +
                        +    @Test(expected = IllegalArgumentException.class)
                        +    public void testThrowIllegalArgumentExceptionWhenTopicSegmentSizeSmallerThanProducerBatchSize() {
                        +        props.put(topicPrefix(TopicConfig.SEGMENT_BYTES_CONFIG), 100);
                        +        props.put(producerPrefix(ProducerConfig.BATCH_SIZE_CONFIG), 101);
                        +        new StreamsConfig(props).getMainConsumerConfigs("groupId", "clientId");
                        +    }
                         
                             static class MisconfiguredSerde implements Serde {
                                 @Override
                        diff --git a/streams/src/test/java/org/apache/kafka/streams/TopologyTest.java b/streams/src/test/java/org/apache/kafka/streams/TopologyTest.java
                        index 8b478852c4116..ece157cd02e73 100644
                        --- a/streams/src/test/java/org/apache/kafka/streams/TopologyTest.java
                        +++ b/streams/src/test/java/org/apache/kafka/streams/TopologyTest.java
                        @@ -174,6 +174,24 @@ public void shouldNotAllowToAddProcessorWithSameName() {
                                 } catch (final TopologyException expected) { }
                             }
                         
                        +    @Test
                        +    public void shouldNotAllowToAddProcessorWithEmptyParents() {
                        +        topology.addSource("source", "topic-1");
                        +        try {
                        +            topology.addProcessor("processor", new MockProcessorSupplier());
                        +            fail("Should throw TopologyException for processor without at least one parent node");
                        +        } catch (final TopologyException expected) { }
                        +    }
                        +
                        +    @Test
                        +    public void shouldNotAllowToAddProcessorWithNullParents() {
                        +        topology.addSource("source", "topic-1");
                        +        try {
                        +            topology.addProcessor("processor", new MockProcessorSupplier(), null);
                        +            fail("Should throw NullPointerException for processor when null parent names are provided");
                        +        } catch (final NullPointerException expected) { }
                        +    }
                        +
                             @Test(expected = TopologyException.class)
                             public void shouldFailOnUnknownSource() {
                                 topology.addProcessor("processor", new MockProcessorSupplier(), "source");
                        @@ -194,6 +212,26 @@ public void shouldNotAllowToAddSinkWithSameName() {
                                 } catch (final TopologyException expected) { }
                             }
                         
                        +    @Test
                        +    public void shouldNotAllowToAddSinkWithEmptyParents() {
                        +        topology.addSource("source", "topic-1");
                        +        topology.addProcessor("processor", new MockProcessorSupplier(), "source");
                        +        try {
                        +            topology.addSink("sink", "topic-2");
                        +            fail("Should throw TopologyException for sink without at least one parent node");
                        +        } catch (final TopologyException expected) { }
                        +    }
                        +
                        +    @Test
                        +    public void shouldNotAllowToAddSinkWithNullParents() {
                        +        topology.addSource("source", "topic-1");
                        +        topology.addProcessor("processor", new MockProcessorSupplier(), "source");
                        +        try {
                        +            topology.addSink("sink", "topic-2", null);
                        +            fail("Should throw NullPointerException for sink when null parent names are provided");
                        +        } catch (final NullPointerException expected) { }
                        +    }
                        +
                             @Test(expected = TopologyException.class)
                             public void shouldFailWithUnknownParent() {
                                 topology.addSink("sink", "topic-2", "source");
                        @@ -236,7 +274,8 @@ public void shouldNotAllowToAddStateStoreToSource() {
                             public void shouldNotAllowToAddStateStoreToSink() {
                                 mockStoreBuilder();
                                 EasyMock.replay(storeBuilder);
                        -        topology.addSink("sink-1", "topic-1");
                        +        topology.addSource("source-1", "topic-1");
                        +        topology.addSink("sink-1", "topic-1", "source-1");
                                 try {
                                     topology.addStateStore(storeBuilder, "sink-1");
                                     fail("Should have thrown TopologyException for adding store to sink node");
                        diff --git a/streams/src/test/java/org/apache/kafka/streams/TopologyTestDriverWrapper.java b/streams/src/test/java/org/apache/kafka/streams/TopologyTestDriverWrapper.java
                        new file mode 100644
                        index 0000000000000..bec4b5f79ffea
                        --- /dev/null
                        +++ b/streams/src/test/java/org/apache/kafka/streams/TopologyTestDriverWrapper.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;
                        +
                        +import org.apache.kafka.streams.errors.StreamsException;
                        +import org.apache.kafka.streams.processor.ProcessorContext;
                        +import org.apache.kafka.streams.processor.internals.ProcessorContextImpl;
                        +import org.apache.kafka.streams.processor.internals.ProcessorNode;
                        +
                        +import java.util.Properties;
                        +
                        +/**
                        + * This class provides access to {@link TopologyTestDriver} protected methods.
                        + * It should only be used for internal testing, in the rare occasions where the
                        + * necessary functionality is not supported by {@link TopologyTestDriver}.
                        + */
                        +public class TopologyTestDriverWrapper extends TopologyTestDriver {
                        +
                        +
                        +    public TopologyTestDriverWrapper(final Topology topology,
                        +                                     final Properties config) {
                        +        super(topology, config);
                        +    }
                        +
                        +    /**
                        +     * Get the processor context, setting the processor whose name is given as current node
                        +     *
                        +     * @param processorName processor name to set as current node
                        +     * @return the processor context
                        +     */
                        +    public ProcessorContext setCurrentNodeForProcessorContext(final String processorName) {
                        +        final ProcessorContext context = task.context();
                        +        ((ProcessorContextImpl) context).setCurrentNode(getProcessor(processorName));
                        +        return context;
                        +    }
                        +
                        +    /**
                        +     * Get a processor by name
                        +     *
                        +     * @param name the name to search for
                        +     * @return the processor matching the search name
                        +     */
                        +    public ProcessorNode getProcessor(final String name) {
                        +        for (final ProcessorNode node : processorTopology.processors()) {
                        +            if (node.name().equals(name)) {
                        +                return node;
                        +            }
                        +        }
                        +        for (final ProcessorNode node : globalTopology.processors()) {
                        +            if (node.name().equals(name)) {
                        +                return node;
                        +            }
                        +        }
                        +        throw new StreamsException("Could not find a processor named '" + name + "'");
                        +    }
                        +}
                        diff --git a/streams/src/test/java/org/apache/kafka/streams/integration/AbstractJoinIntegrationTest.java b/streams/src/test/java/org/apache/kafka/streams/integration/AbstractJoinIntegrationTest.java
                        index 80ab60647adae..b67e3a3f92e87 100644
                        --- a/streams/src/test/java/org/apache/kafka/streams/integration/AbstractJoinIntegrationTest.java
                        +++ b/streams/src/test/java/org/apache/kafka/streams/integration/AbstractJoinIntegrationTest.java
                        @@ -53,6 +53,7 @@
                         import java.util.Iterator;
                         import java.util.List;
                         import java.util.Properties;
                        +import java.util.concurrent.TimeUnit;
                         import java.util.concurrent.atomic.AtomicBoolean;
                         
                         import static org.hamcrest.MatcherAssert.assertThat;
                        @@ -163,7 +164,8 @@ void prepareEnvironment() throws InterruptedException {
                         
                             @After
                             public void cleanup() throws InterruptedException {
                        -        CLUSTER.deleteTopicsAndWait(120000, INPUT_TOPIC_LEFT, INPUT_TOPIC_RIGHT, OUTPUT_TOPIC);
                        +        producer.close(0, TimeUnit.MILLISECONDS);
                        +        CLUSTER.deleteAllTopicsAndWait(120000);
                             }
                         
                             private void checkResult(final String outputTopic, final List expectedResult) throws InterruptedException {
                        diff --git a/streams/src/test/java/org/apache/kafka/streams/integration/AbstractResetIntegrationTest.java b/streams/src/test/java/org/apache/kafka/streams/integration/AbstractResetIntegrationTest.java
                        index 249e2c3cffddf..f1bc112e69ab5 100644
                        --- a/streams/src/test/java/org/apache/kafka/streams/integration/AbstractResetIntegrationTest.java
                        +++ b/streams/src/test/java/org/apache/kafka/streams/integration/AbstractResetIntegrationTest.java
                        @@ -17,12 +17,12 @@
                         package org.apache.kafka.streams.integration;
                         
                         import org.apache.kafka.clients.CommonClientConfigs;
                        -import org.apache.kafka.clients.admin.KafkaAdminClient;
                        +import org.apache.kafka.clients.admin.AdminClient;
                        +import org.apache.kafka.clients.admin.ConsumerGroupDescription;
                         import org.apache.kafka.clients.consumer.ConsumerConfig;
                         import org.apache.kafka.clients.producer.ProducerConfig;
                         import org.apache.kafka.common.config.SslConfigs;
                         import org.apache.kafka.common.config.types.Password;
                        -import org.apache.kafka.common.errors.TimeoutException;
                         import org.apache.kafka.common.serialization.LongDeserializer;
                         import org.apache.kafka.common.serialization.LongSerializer;
                         import org.apache.kafka.common.serialization.Serdes;
                        @@ -61,9 +61,9 @@
                         import java.util.List;
                         import java.util.Map;
                         import java.util.Properties;
                        +import java.util.concurrent.ExecutionException;
                         import java.util.concurrent.TimeUnit;
                         
                        -import kafka.admin.AdminClient;
                         import kafka.tools.StreamsResetter;
                         
                         import static org.hamcrest.CoreMatchers.equalTo;
                        @@ -77,20 +77,15 @@ public abstract class AbstractResetIntegrationTest {
                             private static MockTime mockTime;
                             private static KafkaStreams streams;
                             private static AdminClient adminClient = null;
                        -    private static KafkaAdminClient kafkaAdminClient = null;
                         
                             abstract Map getClientSslConfig();
                         
                             @AfterClass
                             public static void afterClassCleanup() {
                                 if (adminClient != null) {
                        -            adminClient.close();
                        +            adminClient.close(10, TimeUnit.SECONDS);
                                     adminClient = null;
                                 }
                        -        if (kafkaAdminClient != null) {
                        -            kafkaAdminClient.close(10, TimeUnit.SECONDS);
                        -            kafkaAdminClient = null;
                        -        }
                             }
                         
                             private String appID = "abstract-reset-integration-test";
                        @@ -103,9 +98,6 @@ private void prepareEnvironment() {
                                 if (adminClient == null) {
                                     adminClient = AdminClient.create(commonClientConfig);
                                 }
                        -        if (kafkaAdminClient == null) {
                        -            kafkaAdminClient = (KafkaAdminClient) org.apache.kafka.clients.admin.AdminClient.create(commonClientConfig);
                        -        }
                         
                                 boolean timeSet = false;
                                 while (!timeSet) {
                        @@ -178,14 +170,15 @@ private void prepareConfigs() {
                         
                             private static final long STREAMS_CONSUMER_TIMEOUT = 2000L;
                             private static final long CLEANUP_CONSUMER_TIMEOUT = 2000L;
                        -    private static final int TIMEOUT_MULTIPLIER = 5;
                        +    private static final int TIMEOUT_MULTIPLIER = 15;
                         
                             private class ConsumerGroupInactiveCondition implements TestCondition {
                                 @Override
                                 public boolean conditionMet() {
                                     try {
                        -                return adminClient.describeConsumerGroup(appID, 0).consumers().get().isEmpty();
                        -            } catch (final TimeoutException e) {
                        +                ConsumerGroupDescription groupDescription = adminClient.describeConsumerGroups(Collections.singletonList(appID)).describedGroups().get(appID).get();
                        +                return groupDescription.members().isEmpty();
                        +            } catch (final ExecutionException | InterruptedException e) {
                                         return false;
                                     }
                                 }
                        diff --git a/streams/src/test/java/org/apache/kafka/streams/integration/EosIntegrationTest.java b/streams/src/test/java/org/apache/kafka/streams/integration/EosIntegrationTest.java
                        index 30c90c23bb30a..770f579ad983a 100644
                        --- a/streams/src/test/java/org/apache/kafka/streams/integration/EosIntegrationTest.java
                        +++ b/streams/src/test/java/org/apache/kafka/streams/integration/EosIntegrationTest.java
                        @@ -391,8 +391,9 @@ public void shouldNotViolateEosIfOneTaskFailsWithState() throws Exception {
                                 // the app is supposed to emit all 40 update records into the output topic
                                 // the app commits after each 10 records per partition, and thus will have 2*5 uncommitted writes
                                 // and store updates (ie, another 5 uncommitted writes to a changelog topic per partition)
                        +        // in the uncommitted batch sending some data for the new key to validate that upon resuming they will not be shown up in the store
                                 //
                        -        // the failure gets inject after 20 committed and 30 uncommitted records got received
                        +        // the failure gets inject after 20 committed and 10 uncommitted records got received
                                 // -> the failure only kills one thread
                                 // after fail over, we should read 40 committed records and the state stores should contain the correct sums
                                 // per key (even if some records got processed twice)
                        @@ -402,7 +403,7 @@ public void shouldNotViolateEosIfOneTaskFailsWithState() throws Exception {
                                     streams.start();
                         
                                     final List> committedDataBeforeFailure = prepareData(0L, 10L, 0L, 1L);
                        -            final List> uncommittedDataBeforeFailure = prepareData(10L, 15L, 0L, 1L);
                        +            final List> uncommittedDataBeforeFailure = prepareData(10L, 15L, 0L, 1L, 2L, 3L);
                         
                                     final List> dataBeforeFailure = new ArrayList<>();
                                     dataBeforeFailure.addAll(committedDataBeforeFailure);
                        @@ -610,10 +611,6 @@ public void init(final ProcessorContext context) {
                         
                                             @Override
                                             public KeyValue transform(final Long key, final Long value) {
                        -                        if (errorInjected.compareAndSet(true, false)) {
                        -                            // only tries to fail once on one of the task
                        -                            throw new RuntimeException("Injected test exception.");
                        -                        }
                                                 if (gcInjected.compareAndSet(true, false)) {
                                                     while (doGC) {
                                                         try {
                        @@ -631,16 +628,27 @@ public KeyValue transform(final Long key, final Long value) {
                         
                                                 if (state != null) {
                                                     Long sum = state.get(key);
                        +
                                                     if (sum == null) {
                                                         sum = value;
                                                     } else {
                                                         sum += value;
                                                     }
                                                     state.put(key, sum);
                        -                            context.forward(key, sum);
                        -                            return null;
                        +                            state.flush();
                        +                        }
                        +
                        +
                        +                        if (errorInjected.compareAndSet(true, false)) {
                        +                            // only tries to fail once on one of the task
                        +                            throw new RuntimeException("Injected test exception.");
                        +                        }
                        +
                        +                        if (state != null) {
                        +                            return new KeyValue<>(key, state.get(key));
                        +                        } else {
                        +                            return new KeyValue<>(key, value);
                                                 }
                        -                        return new KeyValue<>(key, value);
                                             }
                         
                                             @Override
                        diff --git a/streams/src/test/java/org/apache/kafka/streams/integration/GlobalKTableEOSIntegrationTest.java b/streams/src/test/java/org/apache/kafka/streams/integration/GlobalKTableEOSIntegrationTest.java
                        new file mode 100644
                        index 0000000000000..f7c0e55c05e04
                        --- /dev/null
                        +++ b/streams/src/test/java/org/apache/kafka/streams/integration/GlobalKTableEOSIntegrationTest.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.integration;
                        +
                        +import kafka.utils.MockTime;
                        +import org.apache.kafka.clients.consumer.ConsumerConfig;
                        +import org.apache.kafka.clients.producer.ProducerConfig;
                        +import org.apache.kafka.common.serialization.LongSerializer;
                        +import org.apache.kafka.common.serialization.Serdes;
                        +import org.apache.kafka.common.serialization.StringSerializer;
                        +import org.apache.kafka.common.utils.Bytes;
                        +import org.apache.kafka.streams.kstream.Consumed;
                        +import org.apache.kafka.streams.KafkaStreams;
                        +import org.apache.kafka.streams.KeyValue;
                        +import org.apache.kafka.streams.StreamsBuilder;
                        +import org.apache.kafka.streams.StreamsConfig;
                        +import org.apache.kafka.streams.errors.InvalidStateStoreException;
                        +import org.apache.kafka.streams.integration.utils.EmbeddedKafkaCluster;
                        +import org.apache.kafka.streams.integration.utils.IntegrationTestUtils;
                        +import org.apache.kafka.streams.kstream.ForeachAction;
                        +import org.apache.kafka.streams.kstream.GlobalKTable;
                        +import org.apache.kafka.streams.kstream.KStream;
                        +import org.apache.kafka.streams.kstream.KeyValueMapper;
                        +import org.apache.kafka.streams.kstream.Materialized;
                        +import org.apache.kafka.streams.kstream.ValueJoiner;
                        +import org.apache.kafka.streams.state.KeyValueStore;
                        +import org.apache.kafka.streams.state.QueryableStoreTypes;
                        +import org.apache.kafka.streams.state.ReadOnlyKeyValueStore;
                        +import org.apache.kafka.test.IntegrationTest;
                        +import org.apache.kafka.test.TestCondition;
                        +import org.apache.kafka.test.TestUtils;
                        +import org.junit.After;
                        +import org.junit.Before;
                        +import org.junit.ClassRule;
                        +import org.junit.Test;
                        +import org.junit.experimental.categories.Category;
                        +
                        +import java.io.IOException;
                        +import java.util.Arrays;
                        +import java.util.HashMap;
                        +import java.util.Iterator;
                        +import java.util.Map;
                        +import java.util.Properties;
                        +
                        +@Category({IntegrationTest.class})
                        +public class GlobalKTableEOSIntegrationTest {
                        +    private static final int NUM_BROKERS = 1;
                        +    private static final Properties BROKER_CONFIG;
                        +    static {
                        +        BROKER_CONFIG = new Properties();
                        +        BROKER_CONFIG.put("transaction.state.log.replication.factor", (short) 1);
                        +        BROKER_CONFIG.put("transaction.state.log.min.isr", 1);
                        +    }
                        +
                        +    @ClassRule
                        +    public static final EmbeddedKafkaCluster CLUSTER =
                        +            new EmbeddedKafkaCluster(NUM_BROKERS, BROKER_CONFIG);
                        +
                        +    private static volatile int testNo = 0;
                        +    private final MockTime mockTime = CLUSTER.time;
                        +    private final KeyValueMapper keyMapper = new KeyValueMapper() {
                        +        @Override
                        +        public Long apply(final String key, final Long value) {
                        +            return value;
                        +        }
                        +    };
                        +    private final ValueJoiner joiner = new ValueJoiner() {
                        +        @Override
                        +        public String apply(final Long value1, final String value2) {
                        +            return value1 + "+" + value2;
                        +        }
                        +    };
                        +    private final String globalStore = "globalStore";
                        +    private final Map results = new HashMap<>();
                        +    private StreamsBuilder builder;
                        +    private Properties streamsConfiguration;
                        +    private KafkaStreams kafkaStreams;
                        +    private String globalTableTopic;
                        +    private String streamTopic;
                        +    private GlobalKTable globalTable;
                        +    private KStream stream;
                        +    private ForeachAction foreachAction;
                        +
                        +    @Before
                        +    public void before() throws InterruptedException {
                        +        testNo++;
                        +        builder = new StreamsBuilder();
                        +        createTopics();
                        +        streamsConfiguration = new Properties();
                        +        final String applicationId = "globalTableTopic-table-eos-test-" + testNo;
                        +        streamsConfiguration.put(StreamsConfig.APPLICATION_ID_CONFIG, applicationId);
                        +        streamsConfiguration.put(StreamsConfig.BOOTSTRAP_SERVERS_CONFIG, CLUSTER.bootstrapServers());
                        +        streamsConfiguration.put(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG, "earliest");
                        +        streamsConfiguration.put(StreamsConfig.STATE_DIR_CONFIG, TestUtils.tempDirectory().getPath());
                        +        streamsConfiguration.put(StreamsConfig.CACHE_MAX_BYTES_BUFFERING_CONFIG, 0);
                        +        streamsConfiguration.put(IntegrationTestUtils.INTERNAL_LEAVE_GROUP_ON_CLOSE, true);
                        +        streamsConfiguration.put(StreamsConfig.COMMIT_INTERVAL_MS_CONFIG, 100);
                        +        streamsConfiguration.put(StreamsConfig.PROCESSING_GUARANTEE_CONFIG, "exactly_once");
                        +        globalTable = builder.globalTable(globalTableTopic, Consumed.with(Serdes.Long(), Serdes.String()),
                        +                                          Materialized.>as(globalStore)
                        +                                                  .withKeySerde(Serdes.Long())
                        +                                                  .withValueSerde(Serdes.String()));
                        +        final Consumed stringLongConsumed = Consumed.with(Serdes.String(), Serdes.Long());
                        +        stream = builder.stream(streamTopic, stringLongConsumed);
                        +        foreachAction = new ForeachAction() {
                        +            @Override
                        +            public void apply(final String key, final String value) {
                        +                results.put(key, value);
                        +            }
                        +        };
                        +    }
                        +
                        +    @After
                        +    public void whenShuttingDown() throws IOException {
                        +        if (kafkaStreams != null) {
                        +            kafkaStreams.close();
                        +        }
                        +        IntegrationTestUtils.purgeLocalStreamsState(streamsConfiguration);
                        +    }
                        +
                        +    @Test
                        +    public void shouldKStreamGlobalKTableLeftJoin() throws Exception {
                        +        final KStream streamTableJoin = stream.leftJoin(globalTable, keyMapper, joiner);
                        +        streamTableJoin.foreach(foreachAction);
                        +        produceInitialGlobalTableValues();
                        +        startStreams();
                        +        produceTopicValues(streamTopic);
                        +
                        +        final Map expected = new HashMap<>();
                        +        expected.put("a", "1+A");
                        +        expected.put("b", "2+B");
                        +        expected.put("c", "3+C");
                        +        expected.put("d", "4+D");
                        +        expected.put("e", "5+null");
                        +
                        +        TestUtils.waitForCondition(new TestCondition() {
                        +            @Override
                        +            public boolean conditionMet() {
                        +                return results.equals(expected);
                        +            }
                        +        }, 30000L, "waiting for initial values");
                        +
                        +
                        +        produceGlobalTableValues();
                        +
                        +        final ReadOnlyKeyValueStore replicatedStore = kafkaStreams.store(globalStore, QueryableStoreTypes.keyValueStore());
                        +
                        +        TestUtils.waitForCondition(new TestCondition() {
                        +            @Override
                        +            public boolean conditionMet() {
                        +                return "J".equals(replicatedStore.get(5L));
                        +            }
                        +        }, 30000, "waiting for data in replicated store");
                        +        produceTopicValues(streamTopic);
                        +
                        +        expected.put("a", "1+F");
                        +        expected.put("b", "2+G");
                        +        expected.put("c", "3+H");
                        +        expected.put("d", "4+I");
                        +        expected.put("e", "5+J");
                        +
                        +        TestUtils.waitForCondition(new TestCondition() {
                        +            @Override
                        +            public boolean conditionMet() {
                        +                return results.equals(expected);
                        +            }
                        +        }, 30000L, "waiting for final values");
                        +    }
                        +
                        +    @Test
                        +    public void shouldKStreamGlobalKTableJoin() throws Exception {
                        +        final KStream streamTableJoin = stream.join(globalTable, keyMapper, joiner);
                        +        streamTableJoin.foreach(foreachAction);
                        +        produceInitialGlobalTableValues();
                        +        startStreams();
                        +        produceTopicValues(streamTopic);
                        +
                        +        final Map expected = new HashMap<>();
                        +        expected.put("a", "1+A");
                        +        expected.put("b", "2+B");
                        +        expected.put("c", "3+C");
                        +        expected.put("d", "4+D");
                        +
                        +        TestUtils.waitForCondition(new TestCondition() {
                        +            @Override
                        +            public boolean conditionMet() {
                        +                return results.equals(expected);
                        +            }
                        +        }, 30000L, "waiting for initial values");
                        +
                        +
                        +        produceGlobalTableValues();
                        +
                        +        final ReadOnlyKeyValueStore replicatedStore = kafkaStreams.store(globalStore, QueryableStoreTypes.keyValueStore());
                        +
                        +        TestUtils.waitForCondition(new TestCondition() {
                        +            @Override
                        +            public boolean conditionMet() {
                        +                return "J".equals(replicatedStore.get(5L));
                        +            }
                        +        }, 30000, "waiting for data in replicated store");
                        +
                        +        produceTopicValues(streamTopic);
                        +
                        +        expected.put("a", "1+F");
                        +        expected.put("b", "2+G");
                        +        expected.put("c", "3+H");
                        +        expected.put("d", "4+I");
                        +        expected.put("e", "5+J");
                        +
                        +        TestUtils.waitForCondition(new TestCondition() {
                        +            @Override
                        +            public boolean conditionMet() {
                        +                return results.equals(expected);
                        +            }
                        +        }, 30000L, "waiting for final values");
                        +    }
                        +
                        +    @Test
                        +    public void shouldRestoreTransactionalMessages() throws Exception {
                        +        produceInitialGlobalTableValues();
                        +
                        +        startStreams();
                        +
                        +        final Map expected = new HashMap<>();
                        +        expected.put(1L, "A");
                        +        expected.put(2L, "B");
                        +        expected.put(3L, "C");
                        +        expected.put(4L, "D");
                        +
                        +        TestUtils.waitForCondition(new TestCondition() {
                        +            @Override
                        +            public boolean conditionMet() {
                        +                ReadOnlyKeyValueStore store = null;
                        +                try {
                        +                    store = kafkaStreams.store(globalStore, QueryableStoreTypes.keyValueStore());
                        +                } catch (InvalidStateStoreException ex) {
                        +                    return false;
                        +                }
                        +                Map result = new HashMap<>();
                        +                Iterator> it = store.all();
                        +                while (it.hasNext()) {
                        +                    KeyValue kv = it.next();
                        +                    result.put(kv.key, kv.value);
                        +                }
                        +                return result.equals(expected);
                        +            }
                        +        }, 30000L, "waiting for initial values");
                        +    }
                        +    
                        +    @Test
                        +    public void shouldNotRestoreAbortedMessages() throws Exception {
                        +        produceAbortedMessages();
                        +        produceInitialGlobalTableValues();
                        +        produceAbortedMessages();
                        +
                        +        startStreams();
                        +        
                        +        final Map expected = new HashMap<>();
                        +        expected.put(1L, "A");
                        +        expected.put(2L, "B");
                        +        expected.put(3L, "C");
                        +        expected.put(4L, "D");
                        +
                        +        TestUtils.waitForCondition(new TestCondition() {
                        +            @Override
                        +            public boolean conditionMet() {
                        +                ReadOnlyKeyValueStore store = null;
                        +                try {
                        +                    store = kafkaStreams.store(globalStore, QueryableStoreTypes.keyValueStore());
                        +                } catch (InvalidStateStoreException ex) {
                        +                    return false;
                        +                }
                        +                Map result = new HashMap<>();
                        +                Iterator> it = store.all();
                        +                while (it.hasNext()) {
                        +                    KeyValue kv = it.next();
                        +                    result.put(kv.key, kv.value);
                        +                }
                        +                return result.equals(expected);
                        +            }
                        +        }, 30000L, "waiting for initial values");
                        +    }
                        +
                        +    private void createTopics() throws InterruptedException {
                        +        streamTopic = "stream-" + testNo;
                        +        globalTableTopic = "globalTable-" + testNo;
                        +        CLUSTER.createTopics(streamTopic);
                        +        CLUSTER.createTopic(globalTableTopic, 2, 1);
                        +    }
                        +    
                        +    private void startStreams() {
                        +        kafkaStreams = new KafkaStreams(builder.build(), streamsConfiguration);
                        +        kafkaStreams.start();
                        +    }
                        +
                        +    private void produceTopicValues(final String topic) throws Exception {
                        +        IntegrationTestUtils.produceKeyValuesSynchronously(
                        +                topic,
                        +                Arrays.asList(
                        +                        new KeyValue<>("a", 1L),
                        +                        new KeyValue<>("b", 2L),
                        +                        new KeyValue<>("c", 3L),
                        +                        new KeyValue<>("d", 4L),
                        +                        new KeyValue<>("e", 5L)),
                        +                TestUtils.producerConfig(
                        +                        CLUSTER.bootstrapServers(),
                        +                        StringSerializer.class,
                        +                        LongSerializer.class,
                        +                        new Properties()),
                        +                mockTime);
                        +    }
                        +
                        +    private void produceAbortedMessages() throws Exception {
                        +        final Properties properties = new Properties();
                        +        properties.put(ProducerConfig.TRANSACTIONAL_ID_CONFIG, "someid");
                        +        properties.put(ProducerConfig.RETRIES_CONFIG, 1);
                        +        IntegrationTestUtils.produceAbortedKeyValuesSynchronouslyWithTimestamp(
                        +                globalTableTopic, Arrays.asList(
                        +                        new KeyValue<>(1L, "A"),
                        +                        new KeyValue<>(2L, "B"),
                        +                        new KeyValue<>(3L, "C"),
                        +                        new KeyValue<>(4L, "D")
                        +                        ), 
                        +                TestUtils.producerConfig(
                        +                                CLUSTER.bootstrapServers(),
                        +                                LongSerializer.class,
                        +                                StringSerializer.class,
                        +                                properties),
                        +                mockTime.milliseconds());
                        +    }
                        +
                        +    private void produceInitialGlobalTableValues() throws Exception {
                        +        produceInitialGlobalTableValues(true);
                        +    }
                        +
                        +    private void produceInitialGlobalTableValues(final boolean enableTransactions) throws Exception {
                        +        final Properties properties = new Properties();
                        +        if (enableTransactions) {
                        +            properties.put(ProducerConfig.TRANSACTIONAL_ID_CONFIG, "someid");
                        +            properties.put(ProducerConfig.RETRIES_CONFIG, 1);
                        +        }
                        +        IntegrationTestUtils.produceKeyValuesSynchronously(
                        +                globalTableTopic,
                        +                Arrays.asList(
                        +                        new KeyValue<>(1L, "A"),
                        +                        new KeyValue<>(2L, "B"),
                        +                        new KeyValue<>(3L, "C"),
                        +                        new KeyValue<>(4L, "D")
                        +                        ),
                        +                TestUtils.producerConfig(
                        +                        CLUSTER.bootstrapServers(),
                        +                        LongSerializer.class,
                        +                        StringSerializer.class,
                        +                        properties),
                        +                mockTime,
                        +                enableTransactions);
                        +    }
                        +
                        +    private void produceGlobalTableValues() throws Exception {
                        +        IntegrationTestUtils.produceKeyValuesSynchronously(
                        +                globalTableTopic,
                        +                Arrays.asList(
                        +                        new KeyValue<>(1L, "F"),
                        +                        new KeyValue<>(2L, "G"),
                        +                        new KeyValue<>(3L, "H"),
                        +                        new KeyValue<>(4L, "I"),
                        +                        new KeyValue<>(5L, "J")),
                        +                TestUtils.producerConfig(
                        +                        CLUSTER.bootstrapServers(),
                        +                        LongSerializer.class,
                        +                        StringSerializer.class,
                        +                        new Properties()),
                        +                mockTime);
                        +    }
                        +}
                        diff --git a/streams/src/test/java/org/apache/kafka/streams/integration/GlobalKTableIntegrationTest.java b/streams/src/test/java/org/apache/kafka/streams/integration/GlobalKTableIntegrationTest.java
                        index 8c6a30a5972fe..013e2b68110b7 100644
                        --- a/streams/src/test/java/org/apache/kafka/streams/integration/GlobalKTableIntegrationTest.java
                        +++ b/streams/src/test/java/org/apache/kafka/streams/integration/GlobalKTableIntegrationTest.java
                        @@ -18,19 +18,17 @@
                         
                         import kafka.utils.MockTime;
                         import org.apache.kafka.clients.consumer.ConsumerConfig;
                        -import org.apache.kafka.clients.producer.ProducerConfig;
                         import org.apache.kafka.common.serialization.LongSerializer;
                         import org.apache.kafka.common.serialization.Serdes;
                         import org.apache.kafka.common.serialization.StringSerializer;
                         import org.apache.kafka.common.utils.Bytes;
                        -import org.apache.kafka.streams.kstream.Consumed;
                         import org.apache.kafka.streams.KafkaStreams;
                         import org.apache.kafka.streams.KeyValue;
                         import org.apache.kafka.streams.StreamsBuilder;
                         import org.apache.kafka.streams.StreamsConfig;
                        -import org.apache.kafka.streams.errors.InvalidStateStoreException;
                         import org.apache.kafka.streams.integration.utils.EmbeddedKafkaCluster;
                         import org.apache.kafka.streams.integration.utils.IntegrationTestUtils;
                        +import org.apache.kafka.streams.kstream.Consumed;
                         import org.apache.kafka.streams.kstream.ForeachAction;
                         import org.apache.kafka.streams.kstream.GlobalKTable;
                         import org.apache.kafka.streams.kstream.KStream;
                        @@ -40,6 +38,7 @@
                         import org.apache.kafka.streams.state.KeyValueStore;
                         import org.apache.kafka.streams.state.QueryableStoreTypes;
                         import org.apache.kafka.streams.state.ReadOnlyKeyValueStore;
                        +import org.apache.kafka.streams.state.Stores;
                         import org.apache.kafka.test.IntegrationTest;
                         import org.apache.kafka.test.TestCondition;
                         import org.apache.kafka.test.TestUtils;
                        @@ -52,23 +51,19 @@
                         import java.io.IOException;
                         import java.util.Arrays;
                         import java.util.HashMap;
                        -import java.util.Iterator;
                         import java.util.Map;
                         import java.util.Properties;
                         
                        +import static org.hamcrest.core.IsEqual.equalTo;
                        +import static org.junit.Assert.assertThat;
                        +
                         @Category({IntegrationTest.class})
                         public class GlobalKTableIntegrationTest {
                             private static final int NUM_BROKERS = 1;
                        -    private static final Properties BROKER_CONFIG;
                        -    static {
                        -        BROKER_CONFIG = new Properties();
                        -        BROKER_CONFIG.put("transaction.state.log.replication.factor", (short) 1);
                        -        BROKER_CONFIG.put("transaction.state.log.min.isr", 1);
                        -    }
                         
                             @ClassRule
                             public static final EmbeddedKafkaCluster CLUSTER =
                        -            new EmbeddedKafkaCluster(NUM_BROKERS, BROKER_CONFIG);
                        +            new EmbeddedKafkaCluster(NUM_BROKERS);
                         
                             private static volatile int testNo = 0;
                             private final MockTime mockTime = CLUSTER.time;
                        @@ -231,35 +226,23 @@ public boolean conditionMet() {
                             }
                         
                             @Test
                        -    public void shouldRestoreTransactionalMessages() throws Exception {
                        -        produceInitialGlobalTableValues(true);
                        -        startStreams();
                        +    public void shouldRestoreGlobalInMemoryKTableOnRestart() throws Exception {
                        +        builder = new StreamsBuilder();
                        +        globalTable = builder.globalTable(
                        +            globalTableTopic,
                        +            Consumed.with(Serdes.Long(), Serdes.String()),
                        +            Materialized.as(Stores.inMemoryKeyValueStore(globalStore)));
                         
                        -        final Map expected = new HashMap<>();
                        -        expected.put(1L, "A");
                        -        expected.put(2L, "B");
                        -        expected.put(3L, "C");
                        -        expected.put(4L, "D");
                        +        produceInitialGlobalTableValues();
                         
                        -        TestUtils.waitForCondition(new TestCondition() {
                        -            @Override
                        -            public boolean conditionMet() {
                        -                ReadOnlyKeyValueStore store = null;
                        -                try {
                        -                    store = kafkaStreams.store(globalStore, QueryableStoreTypes.keyValueStore());
                        -                } catch (InvalidStateStoreException ex) {
                        -                    return false;
                        -                }
                        -                Map result = new HashMap<>();
                        -                Iterator> it = store.all();
                        -                while (it.hasNext()) {
                        -                    KeyValue kv = it.next();
                        -                    result.put(kv.key, kv.value);
                        -                }
                        -                return result.equals(expected);
                        -            }
                        -        }, 30000L, "waiting for initial values");
                        -        System.out.println("no failed test");
                        +        startStreams();
                        +        ReadOnlyKeyValueStore store = kafkaStreams.store(globalStore, QueryableStoreTypes.keyValueStore());
                        +        assertThat(store.approximateNumEntries(), equalTo(4L));
                        +        kafkaStreams.close();
                        +
                        +        startStreams();
                        +        store = kafkaStreams.store(globalStore, QueryableStoreTypes.keyValueStore());
                        +        assertThat(store.approximateNumEntries(), equalTo(4L));
                             }
                         
                             private void createTopics() throws InterruptedException {
                        @@ -268,7 +251,7 @@ private void createTopics() throws InterruptedException {
                                 CLUSTER.createTopics(streamTopic);
                                 CLUSTER.createTopic(globalTableTopic, 2, 1);
                             }
                        -
                        +    
                             private void startStreams() {
                                 kafkaStreams = new KafkaStreams(builder.build(), streamsConfiguration);
                                 kafkaStreams.start();
                        @@ -292,29 +275,20 @@ private void produceTopicValues(final String topic) throws Exception {
                             }
                         
                             private void produceInitialGlobalTableValues() throws Exception {
                        -        produceInitialGlobalTableValues(false);
                        -    }
                        -
                        -    private void produceInitialGlobalTableValues(final boolean enableTransactions) throws Exception {
                        -        Properties properties = new Properties();
                        -        if (enableTransactions) {
                        -            properties.put(ProducerConfig.TRANSACTIONAL_ID_CONFIG, "someid");
                        -            properties.put(ProducerConfig.RETRIES_CONFIG, 1);
                        -        }
                                 IntegrationTestUtils.produceKeyValuesSynchronously(
                                         globalTableTopic,
                                         Arrays.asList(
                                                 new KeyValue<>(1L, "A"),
                                                 new KeyValue<>(2L, "B"),
                                                 new KeyValue<>(3L, "C"),
                        -                        new KeyValue<>(4L, "D")),
                        +                        new KeyValue<>(4L, "D")
                        +                        ),
                                         TestUtils.producerConfig(
                                                 CLUSTER.bootstrapServers(),
                                                 LongSerializer.class,
                        -                        StringSerializer.class,
                        -                        properties),
                        -                mockTime,
                        -                enableTransactions);
                        +                        StringSerializer.class
                        +                        ),
                        +                mockTime);
                             }
                         
                             private void produceGlobalTableValues() throws Exception {
                        diff --git a/streams/src/test/java/org/apache/kafka/streams/integration/GlobalThreadShutDownOrderTest.java b/streams/src/test/java/org/apache/kafka/streams/integration/GlobalThreadShutDownOrderTest.java
                        index 0bbea0498da30..b45d689f83144 100644
                        --- a/streams/src/test/java/org/apache/kafka/streams/integration/GlobalThreadShutDownOrderTest.java
                        +++ b/streams/src/test/java/org/apache/kafka/streams/integration/GlobalThreadShutDownOrderTest.java
                        @@ -146,7 +146,7 @@ public void shouldFinishGlobalStoreOperationOnShutDown() throws Exception {
                                     public boolean conditionMet() {
                                         return firstRecordProcessed;
                                     }
                        -        }, 5000L, "Has not processed record within 5 seconds");
                        +        }, 30000, "Has not processed record within 30 seconds");
                         
                                 kafkaStreams.close(30, TimeUnit.SECONDS);
                         
                        diff --git a/streams/src/test/java/org/apache/kafka/streams/integration/InternalTopicIntegrationTest.java b/streams/src/test/java/org/apache/kafka/streams/integration/InternalTopicIntegrationTest.java
                        index d379e0d62f3d0..331906aaa4752 100644
                        --- a/streams/src/test/java/org/apache/kafka/streams/integration/InternalTopicIntegrationTest.java
                        +++ b/streams/src/test/java/org/apache/kafka/streams/integration/InternalTopicIntegrationTest.java
                        @@ -161,7 +161,7 @@ public Iterable apply(final String value) {
                                 //
                                 // Step 3: Verify the state changelog topics are compact
                                 //
                        -        waitForCompletion(streams, 2, 5000);
                        +        waitForCompletion(streams, 2, 30000);
                                 streams.close();
                         
                                 final Properties changelogProps = getTopicProperties(ProcessorStateManager.storeChangelogTopic(appID, "Counts"));
                        @@ -206,7 +206,7 @@ public Iterable apply(final String value) {
                                 //
                                 // Step 3: Verify the state changelog topics are compact
                                 //
                        -        waitForCompletion(streams, 2, 5000);
                        +        waitForCompletion(streams, 2, 30000);
                                 streams.close();
                                 final Properties properties = getTopicProperties(ProcessorStateManager.storeChangelogTopic(appID, "CountWindows"));
                                 final List policies = Arrays.asList(properties.getProperty(LogConfig.CleanupPolicyProp()).split(","));
                        diff --git a/streams/src/test/java/org/apache/kafka/streams/integration/KStreamAggregationIntegrationTest.java b/streams/src/test/java/org/apache/kafka/streams/integration/KStreamAggregationIntegrationTest.java
                        index 10363f82ebe0f..a29332c5da746 100644
                        --- a/streams/src/test/java/org/apache/kafka/streams/integration/KStreamAggregationIntegrationTest.java
                        +++ b/streams/src/test/java/org/apache/kafka/streams/integration/KStreamAggregationIntegrationTest.java
                        @@ -28,7 +28,6 @@
                         import org.apache.kafka.common.serialization.StringDeserializer;
                         import org.apache.kafka.common.serialization.StringSerializer;
                         import org.apache.kafka.common.utils.Bytes;
                        -import org.apache.kafka.streams.kstream.Consumed;
                         import org.apache.kafka.streams.KafkaStreams;
                         import org.apache.kafka.streams.KeyValue;
                         import org.apache.kafka.streams.StreamsBuilder;
                        @@ -36,7 +35,7 @@
                         import org.apache.kafka.streams.integration.utils.EmbeddedKafkaCluster;
                         import org.apache.kafka.streams.integration.utils.IntegrationTestUtils;
                         import org.apache.kafka.streams.kstream.Aggregator;
                        -import org.apache.kafka.streams.kstream.ForeachAction;
                        +import org.apache.kafka.streams.kstream.Consumed;
                         import org.apache.kafka.streams.kstream.Initializer;
                         import org.apache.kafka.streams.kstream.KGroupedStream;
                         import org.apache.kafka.streams.kstream.KStream;
                        @@ -49,16 +48,16 @@
                         import org.apache.kafka.streams.kstream.SessionWindows;
                         import org.apache.kafka.streams.kstream.TimeWindowedDeserializer;
                         import org.apache.kafka.streams.kstream.TimeWindows;
                        +import org.apache.kafka.streams.kstream.Transformer;
                         import org.apache.kafka.streams.kstream.Windowed;
                         import org.apache.kafka.streams.kstream.WindowedSerdes;
                         import org.apache.kafka.streams.kstream.internals.SessionWindow;
                         import org.apache.kafka.streams.kstream.internals.TimeWindow;
                        +import org.apache.kafka.streams.processor.ProcessorContext;
                         import org.apache.kafka.streams.state.KeyValueIterator;
                         import org.apache.kafka.streams.state.KeyValueStore;
                         import org.apache.kafka.streams.state.QueryableStoreTypes;
                         import org.apache.kafka.streams.state.ReadOnlySessionStore;
                        -import org.apache.kafka.streams.state.SessionStore;
                        -import org.apache.kafka.streams.state.WindowStore;
                         import org.apache.kafka.test.IntegrationTest;
                         import org.apache.kafka.test.MockMapper;
                         import org.apache.kafka.test.TestUtils;
                        @@ -136,24 +135,9 @@ public void before() throws InterruptedException {
                                         mapper,
                                         Serialized.with(Serdes.String(), Serdes.String()));
                         
                        -        reducer = new Reducer() {
                        -            @Override
                        -            public String apply(final String value1, final String value2) {
                        -                return value1 + ":" + value2;
                        -            }
                        -        };
                        -        initializer = new Initializer() {
                        -            @Override
                        -            public Integer apply() {
                        -                return 0;
                        -            }
                        -        };
                        -        aggregator = new Aggregator() {
                        -            @Override
                        -            public Integer apply(final String aggKey, final String value, final Integer aggregate) {
                        -                return aggregate + value.length();
                        -            }
                        -        };
                        +        reducer = (value1, value2) -> value1 + ":" + value2;
                        +        initializer = () -> 0;
                        +        aggregator = (aggKey, value, aggregate) -> aggregate + value.length();
                             }
                         
                             @After
                        @@ -181,12 +165,7 @@ public void shouldReduce() throws Exception {
                                     new StringDeserializer(),
                                     10);
                         
                        -        Collections.sort(results, new Comparator>() {
                        -            @Override
                        -            public int compare(final KeyValue o1, final KeyValue o2) {
                        -                return KStreamAggregationIntegrationTest.compare(o1, o2);
                        -            }
                        -        });
                        +        Collections.sort(results, KStreamAggregationIntegrationTest::compare);
                         
                                 assertThat(results, is(Arrays.asList(KeyValue.pair("A", "A"),
                                     KeyValue.pair("A", "A:A"),
                        @@ -218,7 +197,7 @@ public void shouldReduceWindowed() throws Exception {
                                 produceMessages(secondBatchTimestamp);
                                 produceMessages(secondBatchTimestamp);
                         
                        -        Serde> windowedSerde = WindowedSerdes.timeWindowedSerdeFrom(String.class);
                        +        final Serde> windowedSerde = WindowedSerdes.timeWindowedSerdeFrom(String.class);
                                 groupedStream
                                         .windowedBy(TimeWindows.of(500L))
                                         .reduce(reducer)
                        @@ -228,34 +207,28 @@ public void shouldReduceWindowed() throws Exception {
                                 startStreams();
                         
                                 final List, String>> windowedOutput = receiveMessages(
                        -            new TimeWindowedDeserializer(),
                        +            new TimeWindowedDeserializer<>(),
                                     new StringDeserializer(),
                                     String.class,
                                     15);
                         
                                 // read from ConsoleConsumer
                        -        String resultFromConsoleConsumer = readWindowedKeyedMessagesViaConsoleConsumer(
                        -                new TimeWindowedDeserializer(),
                        -                new StringDeserializer(),
                        -                String.class,
                        -                15);
                        +        final String resultFromConsoleConsumer = readWindowedKeyedMessagesViaConsoleConsumer(
                        +            new TimeWindowedDeserializer(),
                        +            new StringDeserializer(),
                        +            String.class,
                        +            15,
                        +            false);
                         
                                 final Comparator, String>>
                                     comparator =
                        -            new Comparator, String>>() {
                        -                @Override
                        -                public int compare(final KeyValue, String> o1,
                        -                                   final KeyValue, String> o2) {
                        -                    final int keyComparison = o1.key.key().compareTo(o2.key.key());
                        -                    return keyComparison == 0 ? o1.value.compareTo(o2.value) : keyComparison;
                        -                }
                        -            };
                        +            Comparator.comparing((KeyValue, String> o) -> o.key.key()).thenComparing(o -> o.value);
                         
                                 Collections.sort(windowedOutput, comparator);
                                 final long firstBatchWindow = firstBatchTimestamp / 500 * 500;
                                 final long secondBatchWindow = secondBatchTimestamp / 500 * 500;
                         
                        -        List, String>> expectResult = Arrays.asList(
                        +        final List, String>> expectResult = Arrays.asList(
                                         new KeyValue<>(new Windowed<>("A", new TimeWindow(firstBatchWindow, Long.MAX_VALUE)), "A"),
                                         new KeyValue<>(new Windowed<>("A", new TimeWindow(secondBatchWindow, Long.MAX_VALUE)), "A"),
                                         new KeyValue<>(new Windowed<>("A", new TimeWindow(secondBatchWindow, Long.MAX_VALUE)), "A:A"),
                        @@ -274,13 +247,13 @@ public int compare(final KeyValue, String> o1,
                                 );
                                 assertThat(windowedOutput, is(expectResult));
                         
                        -        Set expectResultString = new HashSet<>(expectResult.size());
                        -        for (KeyValue, String> eachRecord: expectResult) {
                        +        final Set expectResultString = new HashSet<>(expectResult.size());
                        +        for (final KeyValue, String> eachRecord: expectResult) {
                                     expectResultString.add(eachRecord.toString());
                                 }
                         
                                 // check every message is contained in the expect result
                        -        String[] allRecords = resultFromConsoleConsumer.split("\n");
                        +        final String[] allRecords = resultFromConsoleConsumer.split("\n");
                                 for (String record: allRecords) {
                                     record = "KeyValue(" + record + ")";
                                     assertTrue(expectResultString.contains(record));
                        @@ -306,12 +279,7 @@ public void shouldAggregate() throws Exception {
                                     new IntegerDeserializer(),
                                     10);
                         
                        -        Collections.sort(results, new Comparator>() {
                        -            @Override
                        -            public int compare(final KeyValue o1, final KeyValue o2) {
                        -                return KStreamAggregationIntegrationTest.compare(o1, o2);
                        -            }
                        -        });
                        +        Collections.sort(results, KStreamAggregationIntegrationTest::compare);
                         
                                 assertThat(results, is(Arrays.asList(
                                     KeyValue.pair("A", 1),
                        @@ -336,75 +304,68 @@ public void shouldAggregateWindowed() throws Exception {
                                 produceMessages(secondTimestamp);
                                 produceMessages(secondTimestamp);
                         
                        -        Serde> windowedSerde = WindowedSerdes.timeWindowedSerdeFrom(String.class);
                        +        final Serde> windowedSerde = WindowedSerdes.timeWindowedSerdeFrom(String.class);
                                 groupedStream.windowedBy(TimeWindows.of(500L))
                                         .aggregate(
                                                 initializer,
                                                 aggregator,
                        -                        Materialized.>with(null, Serdes.Integer())
                        +                        Materialized.with(null, Serdes.Integer())
                                         )
                                         .toStream()
                                         .to(outputTopic, Produced.with(windowedSerde, Serdes.Integer()));
                         
                                 startStreams();
                         
                        -        final List, Integer>> windowedMessages = receiveMessages(
                        -            new TimeWindowedDeserializer(),
                        +        final List, KeyValue>> windowedMessages = receiveMessagesWithTimestamp(
                        +            new TimeWindowedDeserializer<>(),
                                     new IntegerDeserializer(),
                                     String.class,
                                     15);
                         
                                 // read from ConsoleConsumer
                        -        String resultFromConsoleConsumer = readWindowedKeyedMessagesViaConsoleConsumer(
                        -                new TimeWindowedDeserializer(),
                        -                new IntegerDeserializer(),
                        -                String.class,
                        -                15);
                        +        final String resultFromConsoleConsumer = readWindowedKeyedMessagesViaConsoleConsumer(
                        +            new TimeWindowedDeserializer(),
                        +            new IntegerDeserializer(),
                        +            String.class,
                        +            15,
                        +            true);
                         
                        -        final Comparator, Integer>>
                        +        final Comparator, KeyValue>>
                                     comparator =
                        -            new Comparator, Integer>>() {
                        -                @Override
                        -                public int compare(final KeyValue, Integer> o1,
                        -                                   final KeyValue, Integer> o2) {
                        -                    final int keyComparison = o1.key.key().compareTo(o2.key.key());
                        -                    return keyComparison == 0 ? o1.value.compareTo(o2.value) : keyComparison;
                        -                }
                        -            };
                        +            Comparator.comparing((KeyValue, KeyValue> o) -> o.key.key()).thenComparingInt(o -> o.value.key);
                         
                                 Collections.sort(windowedMessages, comparator);
                         
                                 final long firstWindow = firstTimestamp / 500 * 500;
                                 final long secondWindow = secondTimestamp / 500 * 500;
                         
                        -        List, Integer>> expectResult = Arrays.asList(
                        -                new KeyValue<>(new Windowed<>("A", new TimeWindow(firstWindow, Long.MAX_VALUE)), 1),
                        -                new KeyValue<>(new Windowed<>("A", new TimeWindow(secondWindow, Long.MAX_VALUE)), 1),
                        -                new KeyValue<>(new Windowed<>("A", new TimeWindow(secondWindow, Long.MAX_VALUE)), 2),
                        -                new KeyValue<>(new Windowed<>("B", new TimeWindow(firstWindow, Long.MAX_VALUE)), 1),
                        -                new KeyValue<>(new Windowed<>("B", new TimeWindow(secondWindow, Long.MAX_VALUE)), 1),
                        -                new KeyValue<>(new Windowed<>("B", new TimeWindow(secondWindow, Long.MAX_VALUE)), 2),
                        -                new KeyValue<>(new Windowed<>("C", new TimeWindow(firstWindow, Long.MAX_VALUE)), 1),
                        -                new KeyValue<>(new Windowed<>("C", new TimeWindow(secondWindow, Long.MAX_VALUE)), 1),
                        -                new KeyValue<>(new Windowed<>("C", new TimeWindow(secondWindow, Long.MAX_VALUE)), 2),
                        -                new KeyValue<>(new Windowed<>("D", new TimeWindow(firstWindow, Long.MAX_VALUE)), 1),
                        -                new KeyValue<>(new Windowed<>("D", new TimeWindow(secondWindow, Long.MAX_VALUE)), 1),
                        -                new KeyValue<>(new Windowed<>("D", new TimeWindow(secondWindow, Long.MAX_VALUE)), 2),
                        -                new KeyValue<>(new Windowed<>("E", new TimeWindow(firstWindow, Long.MAX_VALUE)), 1),
                        -                new KeyValue<>(new Windowed<>("E", new TimeWindow(secondWindow, Long.MAX_VALUE)), 1),
                        -                new KeyValue<>(new Windowed<>("E", new TimeWindow(secondWindow, Long.MAX_VALUE)), 2));
                        +        final List, KeyValue>> expectResult = Arrays.asList(
                        +                new KeyValue<>(new Windowed<>("A", new TimeWindow(firstWindow, Long.MAX_VALUE)), KeyValue.pair(1, firstTimestamp)),
                        +                new KeyValue<>(new Windowed<>("A", new TimeWindow(secondWindow, Long.MAX_VALUE)), KeyValue.pair(1, secondTimestamp)),
                        +                new KeyValue<>(new Windowed<>("A", new TimeWindow(secondWindow, Long.MAX_VALUE)), KeyValue.pair(2, secondTimestamp)),
                        +                new KeyValue<>(new Windowed<>("B", new TimeWindow(firstWindow, Long.MAX_VALUE)), KeyValue.pair(1, firstTimestamp)),
                        +                new KeyValue<>(new Windowed<>("B", new TimeWindow(secondWindow, Long.MAX_VALUE)), KeyValue.pair(1, secondTimestamp)),
                        +                new KeyValue<>(new Windowed<>("B", new TimeWindow(secondWindow, Long.MAX_VALUE)), KeyValue.pair(2, secondTimestamp)),
                        +                new KeyValue<>(new Windowed<>("C", new TimeWindow(firstWindow, Long.MAX_VALUE)), KeyValue.pair(1, firstTimestamp)),
                        +                new KeyValue<>(new Windowed<>("C", new TimeWindow(secondWindow, Long.MAX_VALUE)), KeyValue.pair(1, secondTimestamp)),
                        +                new KeyValue<>(new Windowed<>("C", new TimeWindow(secondWindow, Long.MAX_VALUE)), KeyValue.pair(2, secondTimestamp)),
                        +                new KeyValue<>(new Windowed<>("D", new TimeWindow(firstWindow, Long.MAX_VALUE)), KeyValue.pair(1, firstTimestamp)),
                        +                new KeyValue<>(new Windowed<>("D", new TimeWindow(secondWindow, Long.MAX_VALUE)), KeyValue.pair(1, secondTimestamp)),
                        +                new KeyValue<>(new Windowed<>("D", new TimeWindow(secondWindow, Long.MAX_VALUE)), KeyValue.pair(2, secondTimestamp)),
                        +                new KeyValue<>(new Windowed<>("E", new TimeWindow(firstWindow, Long.MAX_VALUE)), KeyValue.pair(1, firstTimestamp)),
                        +                new KeyValue<>(new Windowed<>("E", new TimeWindow(secondWindow, Long.MAX_VALUE)), KeyValue.pair(1, secondTimestamp)),
                        +                new KeyValue<>(new Windowed<>("E", new TimeWindow(secondWindow, Long.MAX_VALUE)), KeyValue.pair(2, secondTimestamp)));
                         
                                 assertThat(windowedMessages, is(expectResult));
                         
                        -        Set expectResultString = new HashSet<>(expectResult.size());
                        -        for (KeyValue, Integer> eachRecord: expectResult) {
                        -            expectResultString.add(eachRecord.toString());
                        +        final Set expectResultString = new HashSet<>(expectResult.size());
                        +        for (final KeyValue, KeyValue> eachRecord: expectResult) {
                        +            expectResultString.add("CreateTime:" + eachRecord.value.value + ", " + eachRecord.key.toString() + ", " + eachRecord.value.key);
                                 }
                         
                                 // check every message is contained in the expect result
                        -        String[] allRecords = resultFromConsoleConsumer.split("\n");
                        -        for (String record: allRecords) {
                        -            record = "KeyValue(" + record + ")";
                        +        final String[] allRecords = resultFromConsoleConsumer.split("\n");
                        +        for (final String record: allRecords) {
                                     assertTrue(expectResultString.contains(record));
                                 }
                         
                        @@ -419,12 +380,7 @@ private void shouldCountHelper() throws Exception {
                                     new StringDeserializer(),
                                     new LongDeserializer(),
                                     10);
                        -        Collections.sort(results, new Comparator>() {
                        -            @Override
                        -            public int compare(final KeyValue o1, final KeyValue o2) {
                        -                return KStreamAggregationIntegrationTest.compare(o1, o2);
                        -            }
                        -        });
                        +        Collections.sort(results, KStreamAggregationIntegrationTest::compare);
                         
                                 assertThat(results, is(Arrays.asList(
                                     KeyValue.pair("A", 1L),
                        @@ -444,7 +400,7 @@ public int compare(final KeyValue o1, final KeyValue
                             public void shouldCount() throws Exception {
                                 produceMessages(mockTime.milliseconds());
                         
                        -        groupedStream.count(Materialized.>as("count-by-key"))
                        +        groupedStream.count(Materialized.as("count-by-key"))
                                         .toStream()
                                         .to(outputTopic, Produced.with(Serdes.String(), Serdes.Long()));
                         
                        @@ -471,12 +427,7 @@ public void shouldGroupByKey() throws Exception {
                                 stream.groupByKey(Serialized.with(Serdes.Integer(), Serdes.String()))
                                         .windowedBy(TimeWindows.of(500L))
                                         .count()
                        -                .toStream(new KeyValueMapper, Long, String>() {
                        -                    @Override
                        -                    public String apply(final Windowed windowedKey, final Long value) {
                        -                        return windowedKey.key() + "@" + windowedKey.window().start();
                        -                    }
                        -                }).to(outputTopic, Produced.with(Serdes.String(), Serdes.Long()));
                        +                .toStream((windowedKey, value) -> windowedKey.key() + "@" + windowedKey.window().start()).to(outputTopic, Produced.with(Serdes.String(), Serdes.Long()));
                         
                                 startStreams();
                         
                        @@ -484,12 +435,7 @@ public String apply(final Windowed windowedKey, final Long value) {
                                     new StringDeserializer(),
                                     new LongDeserializer(),
                                     10);
                        -        Collections.sort(results, new Comparator>() {
                        -            @Override
                        -            public int compare(final KeyValue o1, final KeyValue o2) {
                        -                return KStreamAggregationIntegrationTest.compare(o1, o2);
                        -            }
                        -        });
                        +        Collections.sort(results, KStreamAggregationIntegrationTest::compare);
                         
                                 final long window = timestamp / 500 * 500;
                                 assertThat(results, is(Arrays.asList(
                        @@ -568,7 +514,7 @@ public void shouldCountSessionWindows() throws Exception {
                                                 new Properties()),
                                         t4);
                         
                        -        final Map, Long> results = new HashMap<>();
                        +        final Map, KeyValue> results = new HashMap<>();
                                 final CountDownLatch latch = new CountDownLatch(11);
                         
                                 builder.stream(userSessionsStream, Consumed.with(Serdes.String(), Serdes.String()))
                        @@ -576,23 +522,34 @@ public void shouldCountSessionWindows() throws Exception {
                                         .windowedBy(SessionWindows.with(sessionGap).until(maintainMillis))
                                         .count()
                                         .toStream()
                        -                .foreach(new ForeachAction, Long>() {
                        -                    @Override
                        -                    public void apply(final Windowed key, final Long value) {
                        -                        results.put(key, value);
                        -                        latch.countDown();
                        -                    }
                        -                });
                        +                .transform(() -> new Transformer, Long, KeyValue>() {
                        +                        private ProcessorContext context;
                        +
                        +                        @Override
                        +                        public void init(final ProcessorContext context) {
                        +                            this.context = context;
                        +                        }
                        +
                        +                        @Override
                        +                        public KeyValue transform(final Windowed key, final Long value) {
                        +                            results.put(key, KeyValue.pair(value, context.timestamp()));
                        +                            latch.countDown();
                        +                            return null;
                        +                        }
                        +
                        +                        @Override
                        +                        public void close() {}
                        +                    });
                         
                                 startStreams();
                                 latch.await(30, TimeUnit.SECONDS);
                        -        assertThat(results.get(new Windowed<>("bob", new SessionWindow(t1, t1))), equalTo(1L));
                        -        assertThat(results.get(new Windowed<>("penny", new SessionWindow(t1, t1))), equalTo(1L));
                        -        assertThat(results.get(new Windowed<>("jo", new SessionWindow(t1, t1))), equalTo(1L));
                        -        assertThat(results.get(new Windowed<>("jo", new SessionWindow(t4, t4))), equalTo(1L));
                        -        assertThat(results.get(new Windowed<>("emily", new SessionWindow(t1, t2))), equalTo(2L));
                        -        assertThat(results.get(new Windowed<>("bob", new SessionWindow(t3, t4))), equalTo(2L));
                        -        assertThat(results.get(new Windowed<>("penny", new SessionWindow(t3, t3))), equalTo(1L));
                        +        assertThat(results.get(new Windowed<>("bob", new SessionWindow(t1, t1))), equalTo(KeyValue.pair(1L, t1)));
                        +        assertThat(results.get(new Windowed<>("penny", new SessionWindow(t1, t1))), equalTo(KeyValue.pair(1L, t1)));
                        +        assertThat(results.get(new Windowed<>("jo", new SessionWindow(t1, t1))), equalTo(KeyValue.pair(1L, t1)));
                        +        assertThat(results.get(new Windowed<>("jo", new SessionWindow(t4, t4))), equalTo(KeyValue.pair(1L, t4)));
                        +        assertThat(results.get(new Windowed<>("emily", new SessionWindow(t1, t2))), equalTo(KeyValue.pair(2L, t2)));
                        +        assertThat(results.get(new Windowed<>("bob", new SessionWindow(t3, t4))), equalTo(KeyValue.pair(2L, t4)));
                        +        assertThat(results.get(new Windowed<>("penny", new SessionWindow(t3, t3))), equalTo(KeyValue.pair(1L, t3)));
                             }
                         
                             @Test
                        @@ -662,25 +619,17 @@ public void shouldReduceSessionWindows() throws Exception {
                                 builder.stream(userSessionsStream, Consumed.with(Serdes.String(), Serdes.String()))
                                         .groupByKey(Serialized.with(Serdes.String(), Serdes.String()))
                                         .windowedBy(SessionWindows.with(sessionGap).until(maintainMillis))
                        -                .reduce(new Reducer() {
                        -                    @Override
                        -                    public String apply(final String value1, final String value2) {
                        -                        return value1 + ":" + value2;
                        -                    }
                        -                }, Materialized.>as(userSessionsStore))
                        +                .reduce((value1, value2) -> value1 + ":" + value2, Materialized.as(userSessionsStore))
                                         .toStream()
                        -                .foreach(new ForeachAction, String>() {
                        -                    @Override
                        -                    public void apply(final Windowed key, final String value) {
                        -                        results.put(key, value);
                        -                        latch.countDown();
                        -                    }
                        +                .foreach((key, value) -> {
                        +                    results.put(key, value);
                        +                    latch.countDown();
                                         });
                         
                                 startStreams();
                                 latch.await(30, TimeUnit.SECONDS);
                                 final ReadOnlySessionStore sessionStore
                        -                = kafkaStreams.store(userSessionsStore, QueryableStoreTypes.sessionStore());
                        +                = kafkaStreams.store(userSessionsStore, QueryableStoreTypes.sessionStore());
                         
                                 // verify correct data received
                                 assertThat(results.get(new Windowed<>("bob", new SessionWindow(t1, t1))), equalTo("start"));
                        @@ -732,16 +681,14 @@ private void startStreams() {
                             }
                         
                             private  List> receiveMessages(final Deserializer keyDeserializer,
                        -                                                        final Deserializer valueDeserializer,
                        -                                                        final int numMessages)
                        +                                                                        final Deserializer valueDeserializer,
                        +                                                                        final int numMessages)
                                 throws InterruptedException {
                                 return receiveMessages(keyDeserializer, valueDeserializer, null, numMessages);
                             }
                         
                        -    private  List> receiveMessages(final Deserializer
                        -                                                                keyDeserializer,
                        -                                                        final Deserializer
                        -                                                                valueDeserializer,
                        +    private  List> receiveMessages(final Deserializer keyDeserializer,
                        +                                                        final Deserializer valueDeserializer,
                                                                                 final Class innerClass,
                                                                                 final int numMessages) throws InterruptedException {
                                 final Properties consumerProperties = new Properties();
                        @@ -761,21 +708,44 @@ private  List> receiveMessages(final Deserializer
                                         60 * 1000);
                             }
                         
                        +    private  List>> receiveMessagesWithTimestamp(final Deserializer keyDeserializer,
                        +                                                                                     final Deserializer valueDeserializer,
                        +                                                                                     final Class innerClass,
                        +                                                                                     final int numMessages) throws InterruptedException {
                        +        final Properties consumerProperties = new Properties();
                        +        consumerProperties.setProperty(ConsumerConfig.BOOTSTRAP_SERVERS_CONFIG, CLUSTER.bootstrapServers());
                        +        consumerProperties.setProperty(ConsumerConfig.GROUP_ID_CONFIG, "kgroupedstream-test-" + testNo);
                        +        consumerProperties.setProperty(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG, "earliest");
                        +        consumerProperties.setProperty(ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG, keyDeserializer.getClass().getName());
                        +        consumerProperties.setProperty(ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG, valueDeserializer.getClass().getName());
                        +        if (keyDeserializer instanceof TimeWindowedDeserializer || keyDeserializer instanceof SessionWindowedDeserializer) {
                        +            consumerProperties.setProperty(StreamsConfig.DEFAULT_WINDOWED_KEY_SERDE_INNER_CLASS,
                        +                Serdes.serdeFrom(innerClass).getClass().getName());
                        +        }
                        +        return IntegrationTestUtils.waitUntilMinKeyValueWithTimestampRecordsReceived(
                        +            consumerProperties,
                        +            outputTopic,
                        +            numMessages,
                        +            60 * 1000);
                        +    }
                        +
                             private  String readWindowedKeyedMessagesViaConsoleConsumer(final Deserializer keyDeserializer,
                                                                                               final Deserializer valueDeserializer,
                                                                                               final Class innerClass,
                        -                                                                      final int numMessages) {
                        -        ByteArrayOutputStream newConsole = new ByteArrayOutputStream();
                        -        PrintStream originalStream = System.out;
                        -        try (PrintStream newStream = new PrintStream(newConsole)) {
                        +                                                                      final int numMessages,
                        +                                                                      final boolean printTimestamp) {
                        +        final ByteArrayOutputStream newConsole = new ByteArrayOutputStream();
                        +        final PrintStream originalStream = System.out;
                        +        try (final PrintStream newStream = new PrintStream(newConsole)) {
                                     System.setOut(newStream);
                         
                        -            String keySeparator = ", ";
                        +            final String keySeparator = ", ";
                                     // manually construct the console consumer argument array
                        -            String[] args = new String[] {
                        +            final String[] args = new String[] {
                                         "--bootstrap-server", CLUSTER.bootstrapServers(),
                                         "--from-beginning",
                                         "--property", "print.key=true",
                        +                "--property", "print.timestamp=" + printTimestamp,
                                         "--topic", outputTopic,
                                         "--max-messages", String.valueOf(numMessages),
                                         "--property", "key.deserializer=" + keyDeserializer.getClass().getName(),
                        diff --git a/streams/src/test/java/org/apache/kafka/streams/integration/QueryableStateIntegrationTest.java b/streams/src/test/java/org/apache/kafka/streams/integration/QueryableStateIntegrationTest.java
                        index ff791be131691..82bf84928581f 100644
                        --- a/streams/src/test/java/org/apache/kafka/streams/integration/QueryableStateIntegrationTest.java
                        +++ b/streams/src/test/java/org/apache/kafka/streams/integration/QueryableStateIntegrationTest.java
                        @@ -190,24 +190,10 @@ public void before() throws Exception {
                                 streamsConfiguration.put(StreamsConfig.COMMIT_INTERVAL_MS_CONFIG, 100);
                                 // override this to make the rebalances happen quickly
                                 streamsConfiguration.put(IntegrationTestUtils.INTERNAL_LEAVE_GROUP_ON_CLOSE, true);
                        +        
                        +        stringComparator = Comparator.comparing((KeyValue o) -> o.key).thenComparing(o -> o.value);
                        +        stringLongComparator = Comparator.comparing((KeyValue o) -> o.key).thenComparingLong(o -> o.value);
                         
                        -
                        -        stringComparator = new Comparator>() {
                        -
                        -            @Override
                        -            public int compare(final KeyValue o1,
                        -                               final KeyValue o2) {
                        -                return o1.key.compareTo(o2.key);
                        -            }
                        -        };
                        -        stringLongComparator = new Comparator>() {
                        -
                        -            @Override
                        -            public int compare(final KeyValue o1,
                        -                               final KeyValue o2) {
                        -                return o1.key.compareTo(o2.key);
                        -            }
                        -        };
                                 inputValues = getInputValues();
                                 inputValuesKeys = new HashSet<>();
                                 for (final String sentence : inputValues) {
                        diff --git a/streams/src/test/java/org/apache/kafka/streams/integration/RegexSourceIntegrationTest.java b/streams/src/test/java/org/apache/kafka/streams/integration/RegexSourceIntegrationTest.java
                        index d03558666b9d8..709063008fef8 100644
                        --- a/streams/src/test/java/org/apache/kafka/streams/integration/RegexSourceIntegrationTest.java
                        +++ b/streams/src/test/java/org/apache/kafka/streams/integration/RegexSourceIntegrationTest.java
                        @@ -58,6 +58,7 @@
                         import java.util.List;
                         import java.util.Map;
                         import java.util.Properties;
                        +import java.util.concurrent.CopyOnWriteArrayList;
                         import java.util.concurrent.atomic.AtomicBoolean;
                         import java.util.regex.Pattern;
                         
                        @@ -146,7 +147,7 @@ public void testRegexMatchesTopicsAWhenCreated() throws Exception {
                                 final KStream pattern1Stream = builder.stream(Pattern.compile("TEST-TOPIC-\\d"));
                         
                                 pattern1Stream.to(DEFAULT_OUTPUT_TOPIC, Produced.with(stringSerde, stringSerde));
                        -        final List assignedTopics = new ArrayList<>();
                        +        final List assignedTopics = new CopyOnWriteArrayList<>();
                                 streams = new KafkaStreams(builder.build(), streamsConfiguration, new DefaultKafkaClientSupplier() {
                                     @Override
                                     public Consumer getConsumer(final Map config) {
                        @@ -195,7 +196,7 @@ public void testRegexMatchesTopicsAWhenDeleted() throws Exception {
                         
                                 pattern1Stream.to(DEFAULT_OUTPUT_TOPIC, Produced.with(stringSerde, stringSerde));
                         
                        -        final List assignedTopics = new ArrayList<>();
                        +        final List assignedTopics = new CopyOnWriteArrayList<>();
                                 streams = new KafkaStreams(builder.build(), streamsConfiguration, new DefaultKafkaClientSupplier() {
                                     @Override
                                     public Consumer getConsumer(final Map config) {
                        @@ -329,8 +330,8 @@ public void testMultipleConsumersCanReadFromPartitionedTopic() throws Exception
                                     partitionedStreamLeader.to(DEFAULT_OUTPUT_TOPIC, Produced.with(stringSerde, stringSerde));
                                     partitionedStreamFollower.to(DEFAULT_OUTPUT_TOPIC, Produced.with(stringSerde, stringSerde));
                         
                        -            final List leaderAssignment = new ArrayList<>();
                        -            final List followerAssignment = new ArrayList<>();
                        +            final List leaderAssignment = new CopyOnWriteArrayList<>();
                        +            final List followerAssignment = new CopyOnWriteArrayList<>();
                         
                                     partitionedStreamsLeader  = new KafkaStreams(builderLeader.build(), streamsConfiguration, new DefaultKafkaClientSupplier() {
                                         @Override
                        diff --git a/streams/src/test/java/org/apache/kafka/streams/integration/RestoreIntegrationTest.java b/streams/src/test/java/org/apache/kafka/streams/integration/RestoreIntegrationTest.java
                        index f6d36f70848b3..dbf85fa46cde8 100644
                        --- a/streams/src/test/java/org/apache/kafka/streams/integration/RestoreIntegrationTest.java
                        +++ b/streams/src/test/java/org/apache/kafka/streams/integration/RestoreIntegrationTest.java
                        @@ -19,7 +19,6 @@
                         import org.apache.kafka.clients.consumer.Consumer;
                         import org.apache.kafka.clients.consumer.ConsumerConfig;
                         import org.apache.kafka.clients.consumer.KafkaConsumer;
                        -import org.apache.kafka.clients.consumer.OffsetAndMetadata;
                         import org.apache.kafka.clients.producer.KafkaProducer;
                         import org.apache.kafka.clients.producer.ProducerConfig;
                         import org.apache.kafka.clients.producer.ProducerRecord;
                        @@ -28,6 +27,7 @@
                         import org.apache.kafka.common.serialization.IntegerSerializer;
                         import org.apache.kafka.common.serialization.Serdes;
                         import org.apache.kafka.common.utils.Bytes;
                        +import org.apache.kafka.common.utils.MockTime;
                         import org.apache.kafka.streams.kstream.Consumed;
                         import org.apache.kafka.streams.KafkaStreams;
                         import org.apache.kafka.streams.KeyValue;
                        @@ -44,11 +44,14 @@
                         import org.apache.kafka.streams.processor.ProcessorContext;
                         import org.apache.kafka.streams.processor.ProcessorSupplier;
                         import org.apache.kafka.streams.processor.StateRestoreListener;
                        +import org.apache.kafka.streams.processor.TaskId;
                        +import org.apache.kafka.streams.processor.internals.StateDirectory;
                         import org.apache.kafka.streams.state.KeyValueBytesStoreSupplier;
                         import org.apache.kafka.streams.state.KeyValueStore;
                         import org.apache.kafka.streams.state.StoreBuilder;
                         import org.apache.kafka.streams.state.Stores;
                         import org.apache.kafka.streams.state.internals.KeyValueStoreBuilder;
                        +import org.apache.kafka.streams.state.internals.OffsetCheckpoint;
                         import org.apache.kafka.test.IntegrationTest;
                         import org.apache.kafka.test.TestUtils;
                         import org.junit.After;
                        @@ -57,10 +60,10 @@
                         import org.junit.Test;
                         import org.junit.experimental.categories.Category;
                         
                        +import java.io.File;
                         import java.util.Arrays;
                        -import java.util.HashMap;
                        +import java.util.Collections;
                         import java.util.List;
                        -import java.util.Map;
                         import java.util.Properties;
                         import java.util.concurrent.CountDownLatch;
                         import java.util.concurrent.ExecutionException;
                        @@ -76,6 +79,8 @@
                         public class RestoreIntegrationTest {
                             private static final int NUM_BROKERS = 1;
                         
                        +    private static final String APPID = "restore-test";
                        +
                             @ClassRule
                             public static final EmbeddedKafkaCluster CLUSTER =
                                     new EmbeddedKafkaCluster(NUM_BROKERS);
                        @@ -83,24 +88,24 @@ public class RestoreIntegrationTest {
                             private static final String INPUT_STREAM_2 = "input-stream-2";
                             private final int numberOfKeys = 10000;
                             private KafkaStreams kafkaStreams;
                        -    private String applicationId = "restore-test";
                        -
                         
                             @BeforeClass
                             public static void createTopics() throws InterruptedException {
                                 CLUSTER.createTopic(INPUT_STREAM, 2, 1);
                                 CLUSTER.createTopic(INPUT_STREAM_2, 2, 1);
                        +        CLUSTER.createTopic(APPID + "-store-changelog", 2, 1);
                             }
                         
                             private Properties props(final String applicationId) {
                                 Properties streamsConfiguration = new Properties();
                                 streamsConfiguration.put(StreamsConfig.APPLICATION_ID_CONFIG, applicationId);
                                 streamsConfiguration.put(StreamsConfig.BOOTSTRAP_SERVERS_CONFIG, CLUSTER.bootstrapServers());
                        -        streamsConfiguration.put(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG, "earliest");
                        +        streamsConfiguration.put(StreamsConfig.CACHE_MAX_BYTES_BUFFERING_CONFIG, 0);
                                 streamsConfiguration.put(StreamsConfig.STATE_DIR_CONFIG, TestUtils.tempDirectory(applicationId).getPath());
                                 streamsConfiguration.put(StreamsConfig.DEFAULT_KEY_SERDE_CLASS_CONFIG, Serdes.Integer().getClass());
                                 streamsConfiguration.put(StreamsConfig.DEFAULT_VALUE_SERDE_CLASS_CONFIG, Serdes.Integer().getClass());
                                 streamsConfiguration.put(StreamsConfig.COMMIT_INTERVAL_MS_CONFIG, 1000);
                        +        streamsConfiguration.put(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG, "earliest");
                                 return streamsConfiguration;
                             }
                         
                        @@ -112,24 +117,106 @@ public void shutdown() {
                             }
                         
                             @Test
                        -    public void shouldRestoreState() throws ExecutionException, InterruptedException {
                        +    public void shouldRestoreStateFromSourceTopic() throws Exception {
                                 final AtomicInteger numReceived = new AtomicInteger(0);
                                 final StreamsBuilder builder = new StreamsBuilder();
                         
                        -        createStateForRestoration();
                        +        final Properties props = props(APPID);
                        +        props.put(StreamsConfig.TOPOLOGY_OPTIMIZATION, StreamsConfig.OPTIMIZE);
                        +
                        +        // restoring from 1000 to 4000 (committed), and then process from 4000 to 5000 on each of the two partitions
                        +        final int offsetLimitDelta = 1000;
                        +        final int offsetCheckpointed = 1000;
                        +        createStateForRestoration(INPUT_STREAM);
                        +        setCommittedOffset(INPUT_STREAM, offsetLimitDelta);
                        +
                        +        final StateDirectory stateDirectory = new StateDirectory(new StreamsConfig(props), new MockTime());
                        +        new OffsetCheckpoint(new File(stateDirectory.directoryForTask(new TaskId(0, 0)), ".checkpoint"))
                        +                .write(Collections.singletonMap(new TopicPartition(INPUT_STREAM, 0), (long) offsetCheckpointed));
                        +        new OffsetCheckpoint(new File(stateDirectory.directoryForTask(new TaskId(0, 1)), ".checkpoint"))
                        +                .write(Collections.singletonMap(new TopicPartition(INPUT_STREAM, 1), (long) offsetCheckpointed));
                        +
                        +        final CountDownLatch startupLatch = new CountDownLatch(1);
                        +        final CountDownLatch shutdownLatch = new CountDownLatch(1);
                         
                                 builder.table(INPUT_STREAM, Consumed.with(Serdes.Integer(), Serdes.Integer()))
                                         .toStream()
                                         .foreach(new ForeachAction() {
                                             @Override
                                             public void apply(final Integer key, final Integer value) {
                        -                        numReceived.incrementAndGet();
                        +                        if (numReceived.incrementAndGet() == 2 * offsetLimitDelta)
                        +                            shutdownLatch.countDown();
                                             }
                                         });
                         
                        +        kafkaStreams = new KafkaStreams(builder.build(), props);
                        +        kafkaStreams.setStateListener(new KafkaStreams.StateListener() {
                        +            @Override
                        +            public void onChange(final KafkaStreams.State newState, final KafkaStreams.State oldState) {
                        +                if (newState == KafkaStreams.State.RUNNING && oldState == KafkaStreams.State.REBALANCING) {
                        +                    startupLatch.countDown();
                        +                }
                        +            }
                        +        });
                        +
                        +        final AtomicLong restored = new AtomicLong(0);
                        +        kafkaStreams.setGlobalStateRestoreListener(new StateRestoreListener() {
                        +            @Override
                        +            public void onRestoreStart(final TopicPartition topicPartition, final String storeName, final long startingOffset, final long endingOffset) {
                        +
                        +            }
                        +
                        +            @Override
                        +            public void onBatchRestored(final TopicPartition topicPartition, final String storeName, final long batchEndOffset, final long numRestored) {
                        +
                        +            }
                        +
                        +            @Override
                        +            public void onRestoreEnd(final TopicPartition topicPartition, final String storeName, final long totalRestored) {
                        +                restored.addAndGet(totalRestored);
                        +            }
                        +        });
                        +        kafkaStreams.start();
                        +
                        +        assertTrue(startupLatch.await(30, TimeUnit.SECONDS));
                        +        assertThat(restored.get(), equalTo((long) numberOfKeys - offsetLimitDelta * 2 - offsetCheckpointed * 2));
                        +
                        +        assertTrue(shutdownLatch.await(30, TimeUnit.SECONDS));
                        +        assertThat(numReceived.get(), equalTo(offsetLimitDelta * 2));
                        +    }
                        +
                        +    @Test
                        +    public void shouldRestoreStateFromChangelogTopic() throws Exception {
                        +        final AtomicInteger numReceived = new AtomicInteger(0);
                        +        final StreamsBuilder builder = new StreamsBuilder();
                        +
                        +        final Properties props = props(APPID);
                        +
                        +        // restoring from 1000 to 5000, and then process from 5000 to 10000 on each of the two partitions
                        +        final int offsetCheckpointed = 1000;
                        +        createStateForRestoration(APPID + "-store-changelog");
                        +        createStateForRestoration(INPUT_STREAM);
                        +
                        +        final StateDirectory stateDirectory = new StateDirectory(new StreamsConfig(props), new MockTime());
                        +        new OffsetCheckpoint(new File(stateDirectory.directoryForTask(new TaskId(0, 0)), ".checkpoint"))
                        +                .write(Collections.singletonMap(new TopicPartition(APPID + "-store-changelog", 0), (long) offsetCheckpointed));
                        +        new OffsetCheckpoint(new File(stateDirectory.directoryForTask(new TaskId(0, 1)), ".checkpoint"))
                        +                .write(Collections.singletonMap(new TopicPartition(APPID + "-store-changelog", 1), (long) offsetCheckpointed));
                         
                                 final CountDownLatch startupLatch = new CountDownLatch(1);
                        -        kafkaStreams = new KafkaStreams(builder.build(), props(applicationId));
                        +        final CountDownLatch shutdownLatch = new CountDownLatch(1);
                        +
                        +        builder.table(INPUT_STREAM, Consumed.with(Serdes.Integer(), Serdes.Integer()), Materialized.as("store"))
                        +                .toStream()
                        +                .foreach(new ForeachAction() {
                        +                    @Override
                        +                    public void apply(final Integer key, final Integer value) {
                        +                        if (numReceived.incrementAndGet() == numberOfKeys)
                        +                            shutdownLatch.countDown();
                        +                    }
                        +                });
                        +
                        +        kafkaStreams = new KafkaStreams(builder.build(), props);
                                 kafkaStreams.setStateListener(new KafkaStreams.StateListener() {
                                     @Override
                                     public void onChange(final KafkaStreams.State newState, final KafkaStreams.State oldState) {
                        @@ -159,8 +246,10 @@ public void onRestoreEnd(final TopicPartition topicPartition, final String store
                                 kafkaStreams.start();
                         
                                 assertTrue(startupLatch.await(30, TimeUnit.SECONDS));
                        -        assertThat(restored.get(), equalTo((long) numberOfKeys));
                        -        assertThat(numReceived.get(), equalTo(0));
                        +        assertThat(restored.get(), equalTo((long) numberOfKeys - 2 * offsetCheckpointed));
                        +
                        +        assertTrue(shutdownLatch.await(30, TimeUnit.SECONDS));
                        +        assertThat(numReceived.get(), equalTo(numberOfKeys));
                             }
                         
                         
                        @@ -178,7 +267,7 @@ public Integer apply(final Integer value1, final Integer value2) {
                                         }, Materialized.>as("reduce-store").withLoggingDisabled());
                         
                                 final CountDownLatch startupLatch = new CountDownLatch(1);
                        -        kafkaStreams = new KafkaStreams(builder.build(), props(applicationId));
                        +        kafkaStreams = new KafkaStreams(builder.build(), props(APPID));
                                 kafkaStreams.setStateListener(new KafkaStreams.StateListener() {
                                     @Override
                                     public void onChange(final KafkaStreams.State newState, final KafkaStreams.State oldState) {
                        @@ -228,7 +317,7 @@ public Processor get() {
                         
                                 final Topology topology = streamsBuilder.build();
                         
                        -        kafkaStreams = new KafkaStreams(topology, props(applicationId + "-logging-disabled"));
                        +        kafkaStreams = new KafkaStreams(topology, props(APPID + "-logging-disabled"));
                         
                                 final CountDownLatch latch = new CountDownLatch(1);
                                 kafkaStreams.setStateListener(new KafkaStreams.StateListener() {
                        @@ -279,8 +368,7 @@ public void close() {
                                 }
                             }
                             
                        -    private void createStateForRestoration()
                        -            throws ExecutionException, InterruptedException {
                        +    private void createStateForRestoration(final String changelogTopic) {
                                 final Properties producerConfig = new Properties();
                                 producerConfig.put(ProducerConfig.BOOTSTRAP_SERVERS_CONFIG, CLUSTER.bootstrapServers());
                         
                        @@ -288,30 +376,33 @@ private void createStateForRestoration()
                                              new KafkaProducer<>(producerConfig, new IntegerSerializer(), new IntegerSerializer())) {
                         
                                     for (int i = 0; i < numberOfKeys; i++) {
                        -                producer.send(new ProducerRecord<>(INPUT_STREAM, i, i));
                        +                producer.send(new ProducerRecord<>(changelogTopic, i, i));
                                     }
                                 }
                        +    }
                         
                        +    private void setCommittedOffset(final String topic, final int limitDelta) {
                                 final Properties consumerConfig = new Properties();
                                 consumerConfig.put(ConsumerConfig.BOOTSTRAP_SERVERS_CONFIG, CLUSTER.bootstrapServers());
                        -        consumerConfig.put(ConsumerConfig.GROUP_ID_CONFIG, applicationId);
                        +        consumerConfig.put(ConsumerConfig.GROUP_ID_CONFIG, APPID);
                        +        consumerConfig.put(ConsumerConfig.CLIENT_ID_CONFIG, "commit-consumer");
                                 consumerConfig.put(ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG, IntegerDeserializer.class);
                                 consumerConfig.put(ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG, IntegerDeserializer.class);
                         
                                 final Consumer consumer = new KafkaConsumer(consumerConfig);
                        -        final List partitions = Arrays.asList(new TopicPartition(INPUT_STREAM, 0),
                        -                                                              new TopicPartition(INPUT_STREAM, 1));
                        +        final List partitions = Arrays.asList(
                        +            new TopicPartition(topic, 0),
                        +            new TopicPartition(topic, 1));
                         
                                 consumer.assign(partitions);
                                 consumer.seekToEnd(partitions);
                         
                        -        final Map offsets = new HashMap<>();
                                 for (TopicPartition partition : partitions) {
                                     final long position = consumer.position(partition);
                        -            offsets.put(partition, new OffsetAndMetadata(position + 1));
                        +            consumer.seek(partition, position - limitDelta);
                                 }
                         
                        -        consumer.commitSync(offsets);
                        +        consumer.commitSync();
                                 consumer.close();
                             }
                         
                        diff --git a/streams/src/test/java/org/apache/kafka/streams/integration/StandbyTaskCreationIntegrationTest.java b/streams/src/test/java/org/apache/kafka/streams/integration/StandbyTaskCreationIntegrationTest.java
                        new file mode 100644
                        index 0000000000000..6ef20bc40b320
                        --- /dev/null
                        +++ b/streams/src/test/java/org/apache/kafka/streams/integration/StandbyTaskCreationIntegrationTest.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.integration;
                        +
                        +import org.apache.kafka.common.serialization.Serdes;
                        +import org.apache.kafka.streams.KafkaStreams;
                        +import org.apache.kafka.streams.KafkaStreams.State;
                        +import org.apache.kafka.streams.KeyValue;
                        +import org.apache.kafka.streams.StreamsBuilder;
                        +import org.apache.kafka.streams.StreamsConfig;
                        +import org.apache.kafka.streams.Topology;
                        +import org.apache.kafka.streams.integration.utils.EmbeddedKafkaCluster;
                        +import org.apache.kafka.streams.kstream.Consumed;
                        +import org.apache.kafka.streams.kstream.Transformer;
                        +import org.apache.kafka.streams.processor.ProcessorContext;
                        +import org.apache.kafka.streams.processor.ThreadMetadata;
                        +import org.apache.kafka.streams.state.KeyValueStore;
                        +import org.apache.kafka.streams.state.StoreBuilder;
                        +import org.apache.kafka.streams.state.Stores;
                        +import org.apache.kafka.test.IntegrationTest;
                        +import org.apache.kafka.test.TestUtils;
                        +import org.junit.After;
                        +import org.junit.BeforeClass;
                        +import org.junit.ClassRule;
                        +import org.junit.Test;
                        +import org.junit.experimental.categories.Category;
                        +
                        +import java.util.Properties;
                        +import java.util.function.Predicate;
                        +
                        +@Category({IntegrationTest.class})
                        +public class StandbyTaskCreationIntegrationTest {
                        +
                        +    private static final int NUM_BROKERS = 1;
                        +
                        +    @ClassRule
                        +    public static final EmbeddedKafkaCluster CLUSTER = new EmbeddedKafkaCluster(NUM_BROKERS);
                        +
                        +    private static final String INPUT_TOPIC = "input-topic";
                        +
                        +    private KafkaStreams client1;
                        +    private KafkaStreams client2;
                        +    private volatile boolean client1IsOk = false;
                        +    private volatile boolean client2IsOk = false;
                        +
                        +    @BeforeClass
                        +    public static void createTopics() throws InterruptedException {
                        +        CLUSTER.createTopic(INPUT_TOPIC, 2, 1);
                        +    }
                        +
                        +    @After
                        +    public void after() {
                        +        client1.close();
                        +        client2.close();
                        +    }
                        +
                        +    private Properties streamsConfiguration() {
                        +        final String applicationId = "testApp";
                        +        final Properties streamsConfiguration = new Properties();
                        +        streamsConfiguration.put(StreamsConfig.APPLICATION_ID_CONFIG, applicationId);
                        +        streamsConfiguration.put(StreamsConfig.BOOTSTRAP_SERVERS_CONFIG, CLUSTER.bootstrapServers());
                        +        streamsConfiguration.put(StreamsConfig.STATE_DIR_CONFIG, TestUtils.tempDirectory(applicationId).getPath());
                        +        streamsConfiguration.put(StreamsConfig.DEFAULT_KEY_SERDE_CLASS_CONFIG, Serdes.Integer().getClass());
                        +        streamsConfiguration.put(StreamsConfig.DEFAULT_VALUE_SERDE_CLASS_CONFIG, Serdes.Integer().getClass());
                        +        streamsConfiguration.put(StreamsConfig.NUM_STANDBY_REPLICAS_CONFIG, 1);
                        +        return streamsConfiguration;
                        +    }
                        +
                        +    @Test
                        +    public void shouldNotCreateAnyStandByTasksForStateStoreWithLoggingDisabled() throws Exception {
                        +        final StreamsBuilder builder = new StreamsBuilder();
                        +        final String stateStoreName = "myTransformState";
                        +        final StoreBuilder> keyValueStoreBuilder =
                        +            Stores.keyValueStoreBuilder(Stores.persistentKeyValueStore(stateStoreName),
                        +                                        Serdes.Integer(),
                        +                                        Serdes.Integer()).withLoggingDisabled();
                        +        builder.addStateStore(keyValueStoreBuilder);
                        +        builder.stream(INPUT_TOPIC, Consumed.with(Serdes.Integer(), Serdes.Integer()))
                        +            .transform(() -> new Transformer>() {
                        +                @SuppressWarnings("unchecked")
                        +                @Override
                        +                public void init(final ProcessorContext context) {}
                        +
                        +                @Override
                        +                public KeyValue transform(final Integer key, final Integer value) {
                        +                    return null;
                        +                }
                        +
                        +                @Override
                        +                public void close() {}
                        +            }, stateStoreName);
                        +
                        +        final Topology topology = builder.build();
                        +        createClients(topology, streamsConfiguration(), topology, streamsConfiguration());
                        +
                        +        setStateListenersForVerification(thread -> thread.standbyTasks().isEmpty() && !thread.activeTasks().isEmpty());
                        +
                        +        startClients();
                        +
                        +        waitUntilBothClientAreOK(
                        +            "At least one client did not reach state RUNNING with active tasks but no stand-by tasks"
                        +        );
                        +    }
                        +
                        +    private void createClients(final Topology topology1,
                        +                               final Properties streamsConfiguration1,
                        +                               final Topology topology2,
                        +                               final Properties streamsConfiguration2) {
                        +
                        +        client1 = new KafkaStreams(topology1, streamsConfiguration1);
                        +        client2 = new KafkaStreams(topology2, streamsConfiguration2);
                        +    }
                        +
                        +    private void setStateListenersForVerification(final Predicate taskCondition) {
                        +        client1.setStateListener((newState, oldState) -> {
                        +            if (newState == State.RUNNING &&
                        +                client1.localThreadsMetadata().stream().allMatch(taskCondition)) {
                        +
                        +                client1IsOk = true;
                        +            }
                        +        });
                        +        client2.setStateListener((newState, oldState) -> {
                        +            if (newState == State.RUNNING &&
                        +                client2.localThreadsMetadata().stream().allMatch(taskCondition)) {
                        +
                        +                client2IsOk = true;
                        +            }
                        +        });
                        +    }
                        +
                        +    private void startClients() {
                        +        client1.start();
                        +        client2.start();
                        +    }
                        +
                        +    private void waitUntilBothClientAreOK(final String message) throws Exception {
                        +        TestUtils.waitForCondition(
                        +            () -> client1IsOk && client2IsOk,
                        +            30 * 1000,
                        +            message + ": "
                        +                + "Client 1 is " + (!client1IsOk ? "NOT " : "") + "OK, "
                        +                + "client 2 is " + (!client2IsOk ? "NOT " : "") + "OK."
                        +        );
                        +    }
                        +}
                        diff --git a/streams/src/test/java/org/apache/kafka/streams/integration/TableTableJoinIntegrationTest.java b/streams/src/test/java/org/apache/kafka/streams/integration/TableTableJoinIntegrationTest.java
                        index b5e6fcb63b948..5fab6660c4e28 100644
                        --- a/streams/src/test/java/org/apache/kafka/streams/integration/TableTableJoinIntegrationTest.java
                        +++ b/streams/src/test/java/org/apache/kafka/streams/integration/TableTableJoinIntegrationTest.java
                        @@ -55,8 +55,8 @@ public void prepareTopology() throws InterruptedException {
                                 appID = "table-table-join-integration-test";
                         
                                 builder = new StreamsBuilder();
                        -        leftTable = builder.table(INPUT_TOPIC_LEFT);
                        -        rightTable = builder.table(INPUT_TOPIC_RIGHT);
                        +        leftTable = builder.table(INPUT_TOPIC_LEFT, Materialized.>as("left").withLoggingDisabled());
                        +        rightTable = builder.table(INPUT_TOPIC_RIGHT, Materialized.>as("right").withLoggingDisabled());
                             }
                         
                             final private String expectedFinalJoinResult = "D-d";
                        diff --git a/streams/src/test/java/org/apache/kafka/streams/integration/utils/EmbeddedKafkaCluster.java b/streams/src/test/java/org/apache/kafka/streams/integration/utils/EmbeddedKafkaCluster.java
                        index 3ea365c59ec82..ab52649dee4bb 100644
                        --- a/streams/src/test/java/org/apache/kafka/streams/integration/utils/EmbeddedKafkaCluster.java
                        +++ b/streams/src/test/java/org/apache/kafka/streams/integration/utils/EmbeddedKafkaCluster.java
                        @@ -137,7 +137,7 @@ private void stop() {
                              * You can use this to e.g. tell Kafka brokers how to connect to this instance.
                              */
                             public String zKConnectString() {
                        -        return "localhost:" + zookeeper.port();
                        +        return "127.0.0.1:" + zookeeper.port();
                             }
                         
                             /**
                        @@ -274,6 +274,24 @@ public void deleteTopicsAndWait(final long timeoutMs, final String... topics) th
                                 }
                             }
                         
                        +    /**
                        +     * Deletes all topics and blocks until all topics got deleted.
                        +     *
                        +     * @param timeoutMs the max time to wait for the topics to be deleted (does not block if {@code <= 0})
                        +     */
                        +    public void deleteAllTopicsAndWait(final long timeoutMs) throws InterruptedException {
                        +        final Set topics = new HashSet<>(JavaConverters.seqAsJavaListConverter(zkUtils.getAllTopics()).asJava());
                        +        for (final String topic : topics) {
                        +            try {
                        +                brokers[0].deleteTopic(topic);
                        +            } catch (final UnknownTopicOrPartitionException e) { }
                        +        }
                        +
                        +        if (timeoutMs > 0) {
                        +            TestUtils.waitForCondition(new TopicsDeletedCondition(topics), timeoutMs, "Topics not deleted after " + timeoutMs + " milli seconds.");
                        +        }
                        +    }
                        +
                             public void deleteAndRecreateTopics(final String... topics) throws InterruptedException {
                                 deleteTopicsAndWait(TOPIC_DELETION_TIMEOUT, topics);
                                 createTopics(topics);
                        @@ -295,6 +313,10 @@ private TopicsDeletedCondition(final String... topics) {
                                     Collections.addAll(deletedTopics, topics);
                                 }
                         
                        +        public TopicsDeletedCondition(final Set topics) {
                        +            deletedTopics.addAll(topics);
                        +        }
                        +
                                 @Override
                                 public boolean conditionMet() {
                                     final Set allTopics = new HashSet<>(
                        diff --git a/streams/src/test/java/org/apache/kafka/streams/integration/utils/IntegrationTestUtils.java b/streams/src/test/java/org/apache/kafka/streams/integration/utils/IntegrationTestUtils.java
                        index fe897c7ac30e6..9146d6f93a260 100644
                        --- a/streams/src/test/java/org/apache/kafka/streams/integration/utils/IntegrationTestUtils.java
                        +++ b/streams/src/test/java/org/apache/kafka/streams/integration/utils/IntegrationTestUtils.java
                        @@ -44,6 +44,7 @@
                         import java.io.File;
                         import java.io.IOException;
                         import java.nio.file.Paths;
                        +import java.time.Duration;
                         import java.util.ArrayList;
                         import java.util.Collection;
                         import java.util.Collections;
                        @@ -51,6 +52,7 @@
                         import java.util.Properties;
                         import java.util.concurrent.ExecutionException;
                         import java.util.concurrent.Future;
                        +import java.util.stream.Collectors;
                         
                         /**
                          * Utility functions to make integration testing more convenient.
                        @@ -163,7 +165,7 @@ public static  void produceKeyValuesSynchronouslyWithTimestamp(final Strin
                                                                                                  final Long timestamp,
                                                                                                  final boolean enabledTransactions)
                                 throws ExecutionException, InterruptedException {
                        -        try (Producer producer = new KafkaProducer<>(producerConfig)) {
                        +        try (final Producer producer = new KafkaProducer<>(producerConfig)) {
                                     if (enabledTransactions) {
                                         producer.initTransactions();
                                         producer.beginTransaction();
                        @@ -179,16 +181,38 @@ public static  void produceKeyValuesSynchronouslyWithTimestamp(final Strin
                                     producer.flush();
                                 }
                             }
                        +    
                        +    public static  void produceAbortedKeyValuesSynchronouslyWithTimestamp(final String topic,
                        +                                                                                final Collection> records,
                        +                                                                                final Properties producerConfig,
                        +                                                                                final Long timestamp)
                        +        throws ExecutionException, InterruptedException {
                        +        try (final Producer producer = new KafkaProducer<>(producerConfig)) {
                        +            producer.initTransactions();
                        +            for (final KeyValue record : records) {
                        +                producer.beginTransaction();
                        +                final Future f = producer
                        +                        .send(new ProducerRecord<>(topic, null, timestamp, record.key, record.value));
                        +                f.get();
                        +                producer.abortTransaction();
                        +            }
                        +        }    
                        +    }
                         
                        -    public static  void produceValuesSynchronously(
                        -        final String topic, final Collection records, final Properties producerConfig, final Time time)
                        +    public static  void produceValuesSynchronously(final String topic,
                        +                                                      final Collection records,
                        +                                                      final Properties producerConfig,
                        +                                                      final Time time)
                                 throws ExecutionException, InterruptedException {
                                 IntegrationTestUtils.produceValuesSynchronously(topic, records, producerConfig, time, false);
                             }
                         
                        -    public static  void produceValuesSynchronously(
                        -        final String topic, final Collection records, final Properties producerConfig, final Time time, final boolean enableTransactions)
                        -        throws ExecutionException, InterruptedException {
                        +    public static  void produceValuesSynchronously(final String topic,
                        +                                                      final Collection records,
                        +                                                      final Properties producerConfig,
                        +                                                      final Time time,
                        +                                                      final boolean enableTransactions)
                        +            throws ExecutionException, InterruptedException {
                                 final Collection> keyedRecords = new ArrayList<>();
                                 for (final V value : records) {
                                     final KeyValue kv = new KeyValue<>(null, value);
                        @@ -240,10 +264,9 @@ public static  List> waitUntilMinRecordsReceived(fina
                             public static  List> waitUntilMinKeyValueRecordsReceived(final Properties consumerConfig,
                                                                                                           final String topic,
                                                                                                           final int expectedNumRecords) throws InterruptedException {
                        -
                                 return waitUntilMinKeyValueRecordsReceived(consumerConfig, topic, expectedNumRecords, DEFAULT_TIMEOUT);
                             }
                        -
                        +    
                             /**
                              * Wait until enough data (key-value records) has been consumed.
                              *
                        @@ -260,14 +283,11 @@ public static  List> waitUntilMinKeyValueRecordsReceived(fi
                                                                                                           final long waitTime) throws InterruptedException {
                                 final List> accumData = new ArrayList<>();
                                 try (final Consumer consumer = createConsumer(consumerConfig)) {
                        -            final TestCondition valuesRead = new TestCondition() {
                        -                @Override
                        -                public boolean conditionMet() {
                        -                    final List> readData =
                        -                        readKeyValues(topic, consumer, waitTime, expectedNumRecords);
                        -                    accumData.addAll(readData);
                        -                    return accumData.size() >= expectedNumRecords;
                        -                }
                        +            final TestCondition valuesRead = () -> {
                        +                final List> readData =
                        +                    readKeyValues(topic, consumer, waitTime, expectedNumRecords);
                        +                accumData.addAll(readData);
                        +                return accumData.size() >= expectedNumRecords;
                                     };
                                     final String conditionDetails = "Did not receive all " + expectedNumRecords + " records from topic " + topic;
                                     TestUtils.waitForCondition(valuesRead, waitTime, conditionDetails);
                        @@ -275,20 +295,83 @@ public boolean conditionMet() {
                                 return accumData;
                             }
                         
                        +    /**
                        +     * Wait until enough data (key-value records) has been consumed.
                        +     *
                        +     * @param consumerConfig     Kafka Consumer configuration
                        +     * @param topic              Topic to consume from
                        +     * @param expectedNumRecords Minimum number of expected records
                        +     * @param waitTime           Upper bound in waiting time in milliseconds
                        +     * @return All the records consumed, or null if no records are consumed
                        +     * @throws AssertionError       if the given wait time elapses
                        +     */
                        +    public static  List>> waitUntilMinKeyValueWithTimestampRecordsReceived(final Properties consumerConfig,
                        +                                                                                                               final String topic,
                        +                                                                                                               final int expectedNumRecords,
                        +                                                                                                               final long waitTime) throws InterruptedException {
                        +        final List>> accumData = new ArrayList<>();
                        +        try (final Consumer consumer = createConsumer(consumerConfig)) {
                        +            final TestCondition valuesRead = () -> {
                        +                final List>> readData =
                        +                    readKeyValuesWithTimestamp(topic, consumer, waitTime, expectedNumRecords);
                        +                accumData.addAll(readData);
                        +                return accumData.size() >= expectedNumRecords;
                        +            };
                        +            final String conditionDetails = "Did not receive all " + expectedNumRecords + " records from topic " + topic;
                        +            TestUtils.waitForCondition(valuesRead, waitTime, conditionDetails);
                        +        }
                        +        return accumData;
                        +    }
                        +
                        +    public static  List> waitUntilFinalKeyValueRecordsReceived(final Properties consumerConfig,
                        +                                                                                    final String topic,
                        +                                                                                    final List> expectedRecords) throws InterruptedException {
                        +        return waitUntilFinalKeyValueRecordsReceived(consumerConfig, topic, expectedRecords, DEFAULT_TIMEOUT);
                        +    }
                        +
                        +    public static  List> waitUntilFinalKeyValueRecordsReceived(final Properties consumerConfig,
                        +                                                                                    final String topic,
                        +                                                                                    final List> expectedRecords,
                        +                                                                                    final long waitTime) throws InterruptedException {
                        +        final List> accumData = new ArrayList<>();
                        +        try (final Consumer consumer = createConsumer(consumerConfig)) {
                        +            final TestCondition valuesRead = () -> {
                        +                final List> readData =
                        +                    readKeyValues(topic, consumer, waitTime, expectedRecords.size());
                        +                accumData.addAll(readData);
                        +
                        +                final int accumLastIndex = accumData.size() - 1;
                        +                final int expectedLastIndex = expectedRecords.size() - 1;
                        +
                        +                // filter out all intermediate records we don't want
                        +                final List> accumulatedActual = accumData.stream().filter(expectedRecords::contains).collect(Collectors.toList());
                        +
                        +                // need this check as filtering above could have removed the last record from accumData, but it did not
                        +                // equal the last expected record
                        +                final boolean lastRecordsMatch = accumData.get(accumLastIndex).equals(expectedRecords.get(expectedLastIndex));
                        +
                        +                // returns true only if the remaining records in both lists are the same and in the same order
                        +                // and the last record received matches the last expected record
                        +                return accumulatedActual.equals(expectedRecords) && lastRecordsMatch;
                        +
                        +            };
                        +            final String conditionDetails = "Did not receive all " + expectedRecords + " records from topic " + topic;
                        +            TestUtils.waitForCondition(valuesRead, waitTime, conditionDetails);
                        +        }
                        +        return accumData;
                        +    }
                        +
                             public static  List> waitUntilMinRecordsReceived(final Properties consumerConfig,
                                                                                                         final String topic,
                                                                                                         final int expectedNumRecords,
                                                                                                         final long waitTime) throws InterruptedException {
                                 final List> accumData = new ArrayList<>();
                                 try (final Consumer consumer = createConsumer(consumerConfig)) {
                        -            final TestCondition valuesRead = new TestCondition() {
                        -                @Override
                        -                public boolean conditionMet() {
                        -                    final List> readData =
                        -                        readRecords(topic, consumer, waitTime, expectedNumRecords);
                        -                    accumData.addAll(readData);
                        -                    return accumData.size() >= expectedNumRecords;
                        -                }
                        +            final TestCondition valuesRead = () -> {
                        +                final List> readData =
                        +                    readRecords(topic, consumer, waitTime, expectedNumRecords);
                        +                accumData.addAll(readData);
                        +                return accumData.size() >= expectedNumRecords;
                                     };
                                     final String conditionDetails = "Did not receive all " + expectedNumRecords + " records from topic " + topic;
                                     TestUtils.waitForCondition(valuesRead, waitTime, conditionDetails);
                        @@ -319,14 +402,11 @@ public static  List waitUntilMinValuesRecordsReceived(final Properties con
                                                                                         final long waitTime) throws InterruptedException {
                                 final List accumData = new ArrayList<>();
                                 try (final Consumer consumer = createConsumer(consumerConfig)) {
                        -            final TestCondition valuesRead = new TestCondition() {
                        -                @Override
                        -                public boolean conditionMet() {
                        -                    final List readData =
                        -                        readValues(topic, consumer, waitTime, expectedNumRecords);
                        -                    accumData.addAll(readData);
                        -                    return accumData.size() >= expectedNumRecords;
                        -                }
                        +            final TestCondition valuesRead = () -> {
                        +                final List readData =
                        +                    readValues(topic, consumer, waitTime, expectedNumRecords);
                        +                accumData.addAll(readData);
                        +                return accumData.size() >= expectedNumRecords;
                                     };
                                     final String conditionDetails = "Did not receive all " + expectedNumRecords + " records from topic " + topic;
                                     TestUtils.waitForCondition(valuesRead, waitTime, conditionDetails);
                        @@ -351,23 +431,20 @@ public static void waitUntilMetadataIsPropagated(final List servers
                                                                              final String topic,
                                                                              final int partition,
                                                                              final long timeout) throws InterruptedException {
                        -        TestUtils.waitForCondition(new TestCondition() {
                        -            @Override
                        -            public boolean conditionMet() {
                        -                for (final KafkaServer server : servers) {
                        -                    final MetadataCache metadataCache = server.apis().metadataCache();
                        -                    final Option partitionInfo =
                        -                            metadataCache.getPartitionInfo(topic, partition);
                        -                    if (partitionInfo.isEmpty()) {
                        -                        return false;
                        -                    }
                        -                    final UpdateMetadataRequest.PartitionState metadataPartitionState = partitionInfo.get();
                        -                    if (!Request.isValidBrokerId(metadataPartitionState.basePartitionState.leader)) {
                        -                        return false;
                        -                    }
                        +        TestUtils.waitForCondition(() -> {
                        +            for (final KafkaServer server : servers) {
                        +                final MetadataCache metadataCache = server.apis().metadataCache();
                        +                final Option partitionInfo =
                        +                        metadataCache.getPartitionInfo(topic, partition);
                        +                if (partitionInfo.isEmpty()) {
                        +                    return false;
                        +                }
                        +                final UpdateMetadataRequest.PartitionState metadataPartitionState = partitionInfo.get();
                        +                if (!Request.isValidBrokerId(metadataPartitionState.basePartitionState.leader)) {
                        +                    return false;
                                         }
                        -                return true;
                                     }
                        +            return true;
                                 }, timeout, "metadata for topic=" + topic + " partition=" + partition + " not propagated to all brokers");
                         
                             }
                        @@ -452,6 +529,28 @@ private static  List> readKeyValues(final String topic,
                                 return consumedValues;
                             }
                         
                        +    /**
                        +     * Returns up to `maxMessages` by reading via the provided consumer (the topic(s) to read from
                        +     * are already configured in the consumer).
                        +     *
                        +     * @param topic          Kafka topic to read messages from
                        +     * @param consumer       Kafka consumer
                        +     * @param waitTime       Maximum wait time in milliseconds
                        +     * @param maxMessages    Maximum number of messages to read via the consumer
                        +     * @return The KeyValue elements retrieved via the consumer
                        +     */
                        +    private static  List>> readKeyValuesWithTimestamp(final String topic,
                        +                                                                                          final Consumer consumer,
                        +                                                                                          final long waitTime,
                        +                                                                                          final int maxMessages) {
                        +        final List>> consumedValues = new ArrayList<>();
                        +        final List> records = readRecords(topic, consumer, waitTime, maxMessages);
                        +        for (final ConsumerRecord record : records) {
                        +            consumedValues.add(new KeyValue<>(record.key(), KeyValue.pair(record.value(), record.timestamp())));
                        +        }
                        +        return consumedValues;
                        +    }
                        +
                             private static  List> readRecords(final String topic,
                                                                                          final Consumer consumer,
                                                                                          final long waitTime,
                        @@ -464,7 +563,7 @@ private static  List> readRecords(final String topic,
                                 while (totalPollTimeMs < waitTime &&
                                     continueConsuming(consumerRecords.size(), maxMessages)) {
                                     totalPollTimeMs += pollIntervalMs;
                        -            final ConsumerRecords records = consumer.poll(pollIntervalMs);
                        +            final ConsumerRecords records = consumer.poll(Duration.ofMillis(pollIntervalMs));
                         
                                     for (final ConsumerRecord record : records) {
                                         consumerRecords.add(record);
                        diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/WindowedSerdesTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/WindowedSerdesTest.java
                        new file mode 100644
                        index 0000000000000..4360d0846ec1f
                        --- /dev/null
                        +++ b/streams/src/test/java/org/apache/kafka/streams/kstream/WindowedSerdesTest.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;
                        +
                        +import org.apache.kafka.common.serialization.Serde;
                        +import org.apache.kafka.streams.kstream.internals.SessionWindow;
                        +import org.apache.kafka.streams.kstream.internals.TimeWindow;
                        +import org.junit.Assert;
                        +import org.junit.Test;
                        +
                        +public class WindowedSerdesTest {
                        +
                        +    private final String topic = "sample";
                        +
                        +    @Test
                        +    public void testTimeWindowSerdeFrom() {
                        +        final Windowed timeWindowed = new Windowed<>(10, new TimeWindow(0, Long.MAX_VALUE));
                        +        final Serde> timeWindowedSerde = WindowedSerdes.timeWindowedSerdeFrom(Integer.class);
                        +        final byte[] bytes = timeWindowedSerde.serializer().serialize(topic, timeWindowed);
                        +        final Windowed windowed = timeWindowedSerde.deserializer().deserialize(topic, bytes);
                        +        Assert.assertEquals(timeWindowed, windowed);
                        +    }
                        +
                        +    @Test
                        +    public void testSessionWindowedSerdeFrom() {
                        +        final Windowed sessionWindowed = new Windowed<>(10, new SessionWindow(0, 1));
                        +        final Serde> sessionWindowedSerde = WindowedSerdes.sessionWindowedSerdeFrom(Integer.class);
                        +        final byte[] bytes = sessionWindowedSerde.serializer().serialize(topic, sessionWindowed);
                        +        final Windowed windowed = sessionWindowedSerde.deserializer().deserialize(topic, bytes);
                        +        Assert.assertEquals(sessionWindowed, windowed);
                        +    }
                        +
                        +}
                        diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/InternalStreamsBuilderTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/InternalStreamsBuilderTest.java
                        index 63432ffc43927..ef3fcd6110f7f 100644
                        --- a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/InternalStreamsBuilderTest.java
                        +++ b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/InternalStreamsBuilderTest.java
                        @@ -137,7 +137,7 @@ public void shouldStillMaterializeSourceKTableIfMaterializedIsntQueryable() {
                                 assertEquals(storeName, topology.stateStores().get(0).name());
                         
                                 assertEquals(1, topology.storeToChangelogTopic().size());
                        -        assertEquals("topic2", topology.storeToChangelogTopic().get(storeName));
                        +        assertEquals("app-id-prefix-STATE-STORE-0000000000-changelog", topology.storeToChangelogTopic().get(storeName));
                                 assertNull(table1.queryableStoreName());
                             }
                             
                        diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KGroupedTableImplTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KGroupedTableImplTest.java
                        index 4b8298fb2fab8..79e0b42226dab 100644
                        --- a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KGroupedTableImplTest.java
                        +++ b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KGroupedTableImplTest.java
                        @@ -199,7 +199,7 @@ public KeyValue apply(String key, Number value) {
                                 final Map results = getReducedResults(reduced);
                                 try (final TopologyTestDriver driver = new TopologyTestDriver(builder.build(), props)) {
                                     assertReduced(results, topic, driver);
                        -            final KeyValueStore reduce = (KeyValueStore) driver.getStateStore("reduce");
                        +            final KeyValueStore reduce = driver.getKeyValueStore("reduce");
                                     assertThat(reduce.get("A"), equalTo(5));
                                     assertThat(reduce.get("B"), equalTo(6));
                                 }
                        @@ -240,7 +240,7 @@ public void shouldAggregateAndMaterializeResults() {
                         
                                 try (final TopologyTestDriver driver = new TopologyTestDriver(builder.build(), props)) {
                                     processData(topic, driver);
                        -            final KeyValueStore aggregate = (KeyValueStore) driver.getStateStore("aggregate");
                        +            final KeyValueStore aggregate = driver.getKeyValueStore("aggregate");
                                     assertThat(aggregate.get("1"), equalTo("0+1+1+1"));
                                     assertThat(aggregate.get("2"), equalTo("0+2+2"));
                                 }
                        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
                        index 7aed8e1788fbe..0ee9d5e010b99 100644
                        --- 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
                        @@ -55,12 +55,15 @@
                         import java.util.List;
                         import java.util.Properties;
                         import java.util.concurrent.TimeUnit;
                        +import java.util.regex.Matcher;
                         import java.util.regex.Pattern;
                         
                         import static org.hamcrest.CoreMatchers.equalTo;
                        +import static org.hamcrest.CoreMatchers.notNullValue;
                         import static org.hamcrest.core.IsInstanceOf.instanceOf;
                         import static org.junit.Assert.assertEquals;
                         import static org.junit.Assert.assertThat;
                        +import static org.junit.Assert.assertTrue;
                         import static org.junit.Assert.fail;
                         
                         
                        @@ -276,6 +279,27 @@ public void shouldUseRecordMetadataTimestampExtractorWhenInternalRepartitioningT
                                     }
                                 }
                             }
                        +
                        +    @Test
                        +    public void shouldPropagateRepartitionFlagAfterGlobalKTableJoin() {
                        +        final StreamsBuilder builder = new StreamsBuilder();
                        +        final GlobalKTable globalKTable = builder.globalTable("globalTopic");
                        +        final KeyValueMapper kvMappper = (k, v) -> k + v;
                        +        final ValueJoiner valueJoiner = (v1, v2) -> v1 + v2;
                        +        builder.stream("topic").selectKey((k, v) -> v)
                        +            .join(globalKTable, kvMappper, valueJoiner)
                        +            .groupByKey()
                        +            .count();
                        +
                        +        final Pattern repartitionTopicPattern = Pattern.compile("Sink: .*-repartition");
                        +        final String topology = builder.build().describe().toString();
                        +        final Matcher matcher = repartitionTopicPattern.matcher(topology);
                        +        assertTrue(matcher.find());
                        +        final String match = matcher.group();
                        +        assertThat(match, notNullValue());
                        +        assertTrue(match.endsWith("repartition"));
                        +
                        +    }
                             
                             @Test
                             public void testToWithNullValueSerdeDoesntNPE() {
                        diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KTableFilterTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KTableFilterTest.java
                        index c37078df99c35..2cf192b9f4589 100644
                        --- a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KTableFilterTest.java
                        +++ b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KTableFilterTest.java
                        @@ -16,45 +16,40 @@
                          */
                         package org.apache.kafka.streams.kstream.internals;
                         
                        -import org.apache.kafka.common.serialization.Serde;
                        +import org.apache.kafka.common.serialization.IntegerSerializer;
                         import org.apache.kafka.common.serialization.Serdes;
                        +import org.apache.kafka.common.serialization.StringSerializer;
                         import org.apache.kafka.common.utils.Bytes;
                        -import org.apache.kafka.streams.kstream.Consumed;
                         import org.apache.kafka.streams.StreamsBuilder;
                         import org.apache.kafka.streams.Topology;
                        +import org.apache.kafka.streams.TopologyTestDriver;
                        +import org.apache.kafka.streams.TopologyTestDriverWrapper;
                        +import org.apache.kafka.streams.TopologyWrapper;
                        +import org.apache.kafka.streams.kstream.Consumed;
                         import org.apache.kafka.streams.kstream.KTable;
                         import org.apache.kafka.streams.kstream.Materialized;
                         import org.apache.kafka.streams.kstream.Predicate;
                        +import org.apache.kafka.streams.processor.internals.InternalTopologyBuilder;
                         import org.apache.kafka.streams.state.KeyValueStore;
                        -import org.apache.kafka.test.KStreamTestDriver;
                        +import org.apache.kafka.streams.test.ConsumerRecordFactory;
                        +import org.apache.kafka.test.MockMapper;
                         import org.apache.kafka.test.MockProcessor;
                         import org.apache.kafka.test.MockProcessorSupplier;
                         import org.apache.kafka.test.MockReducer;
                        -import org.apache.kafka.test.MockMapper;
                        -import org.apache.kafka.test.TestUtils;
                        -import org.junit.Before;
                        -import org.junit.Rule;
                        +import org.apache.kafka.test.StreamsTestUtils;
                         import org.junit.Test;
                         
                        -import java.io.File;
                         import java.util.List;
                        +import java.util.Properties;
                         
                         import static org.junit.Assert.assertEquals;
                         import static org.junit.Assert.assertNull;
                         
                         public class KTableFilterTest {
                         
                        -    final private Serde intSerde = Serdes.Integer();
                        -    final private Serde stringSerde = Serdes.String();
                        -    private final Consumed consumed = Consumed.with(stringSerde, intSerde);
                        -    @Rule
                        -    public final KStreamTestDriver driver = new KStreamTestDriver();
                        -    private File stateDir = null;
                        -
                        -    @Before
                        -    public void setUp() {
                        -        stateDir = TestUtils.tempDirectory("kafka-test");
                        -    }
                        +    private final Consumed consumed = Consumed.with(Serdes.String(), Serdes.Integer());
                        +    private final ConsumerRecordFactory recordFactory = new ConsumerRecordFactory<>(new StringSerializer(), new IntegerSerializer());
                        +    private final Properties props = StreamsTestUtils.topologyTestConfig(Serdes.String(), Serdes.Integer());
                         
                             private void doTestKTable(final StreamsBuilder builder,
                                                       final KTable table2,
                        @@ -64,16 +59,14 @@ private void doTestKTable(final StreamsBuilder builder,
                                 table2.toStream().process(supplier);
                                 table3.toStream().process(supplier);
                         
                        -        driver.setUp(builder, stateDir, Serdes.String(), Serdes.Integer());
                        -
                        -        driver.process(topic, "A", 1);
                        -        driver.process(topic, "B", 2);
                        -        driver.process(topic, "C", 3);
                        -        driver.process(topic, "D", 4);
                        -        driver.flushState();
                        -        driver.process(topic, "A", null);
                        -        driver.process(topic, "B", null);
                        -        driver.flushState();
                        +        try (final TopologyTestDriver driver = new TopologyTestDriver(builder.build(), props)) {
                        +            driver.pipeInput(recordFactory.create(topic, "A", 1));
                        +            driver.pipeInput(recordFactory.create(topic, "B", 2));
                        +            driver.pipeInput(recordFactory.create(topic, "C", 3));
                        +            driver.pipeInput(recordFactory.create(topic, "D", 4));
                        +            driver.pipeInput(recordFactory.create(topic, "A", null));
                        +            driver.pipeInput(recordFactory.create(topic, "B", null));
                        +        }
                         
                                 final List> processors = supplier.capturedProcessors(2);
                         
                        @@ -136,63 +129,68 @@ private void doTestValueGetter(final StreamsBuilder builder,
                                                            final KTableImpl table2,
                                                            final KTableImpl table3,
                                                            final String topic1) {
                        +
                        +        final Topology topology = builder.build();
                        +
                                 KTableValueGetterSupplier getterSupplier2 = table2.valueGetterSupplier();
                                 KTableValueGetterSupplier getterSupplier3 = table3.valueGetterSupplier();
                         
                        -        driver.setUp(builder, stateDir, Serdes.String(), Serdes.Integer());
                        +        final InternalTopologyBuilder topologyBuilder = TopologyWrapper.getInternalTopologyBuilder(topology);
                        +        topologyBuilder.connectProcessorAndStateStores(table2.name, getterSupplier2.storeNames());
                        +        topologyBuilder.connectProcessorAndStateStores(table3.name, getterSupplier3.storeNames());
                         
                        -        KTableValueGetter getter2 = getterSupplier2.get();
                        -        KTableValueGetter getter3 = getterSupplier3.get();
                        +        try (final TopologyTestDriverWrapper driver = new TopologyTestDriverWrapper(topology, props)) {
                         
                        -        getter2.init(driver.context());
                        -        getter3.init(driver.context());
                        +            KTableValueGetter getter2 = getterSupplier2.get();
                        +            KTableValueGetter getter3 = getterSupplier3.get();
                         
                        -        driver.process(topic1, "A", 1);
                        -        driver.process(topic1, "B", 1);
                        -        driver.process(topic1, "C", 1);
                        +            getter2.init(driver.setCurrentNodeForProcessorContext(table2.name));
                        +            getter3.init(driver.setCurrentNodeForProcessorContext(table3.name));
                         
                        -        assertNull(getter2.get("A"));
                        -        assertNull(getter2.get("B"));
                        -        assertNull(getter2.get("C"));
                        +            driver.pipeInput(recordFactory.create(topic1, "A", 1));
                        +            driver.pipeInput(recordFactory.create(topic1, "B", 1));
                        +            driver.pipeInput(recordFactory.create(topic1, "C", 1));
                         
                        -        assertEquals(1, (int) getter3.get("A"));
                        -        assertEquals(1, (int) getter3.get("B"));
                        -        assertEquals(1, (int) getter3.get("C"));
                        +            assertNull(getter2.get("A"));
                        +            assertNull(getter2.get("B"));
                        +            assertNull(getter2.get("C"));
                         
                        -        driver.process(topic1, "A", 2);
                        -        driver.process(topic1, "B", 2);
                        -        driver.flushState();
                        +            assertEquals(1, (int) getter3.get("A"));
                        +            assertEquals(1, (int) getter3.get("B"));
                        +            assertEquals(1, (int) getter3.get("C"));
                         
                        -        assertEquals(2, (int) getter2.get("A"));
                        -        assertEquals(2, (int) getter2.get("B"));
                        -        assertNull(getter2.get("C"));
                        +            driver.pipeInput(recordFactory.create(topic1, "A", 2));
                        +            driver.pipeInput(recordFactory.create(topic1, "B", 2));
                         
                        -        assertNull(getter3.get("A"));
                        -        assertNull(getter3.get("B"));
                        -        assertEquals(1, (int) getter3.get("C"));
                        +            assertEquals(2, (int) getter2.get("A"));
                        +            assertEquals(2, (int) getter2.get("B"));
                        +            assertNull(getter2.get("C"));
                         
                        -        driver.process(topic1, "A", 3);
                        -        driver.flushState();
                        +            assertNull(getter3.get("A"));
                        +            assertNull(getter3.get("B"));
                        +            assertEquals(1, (int) getter3.get("C"));
                         
                        -        assertNull(getter2.get("A"));
                        -        assertEquals(2, (int) getter2.get("B"));
                        -        assertNull(getter2.get("C"));
                        +            driver.pipeInput(recordFactory.create(topic1, "A", 3));
                         
                        -        assertEquals(3, (int) getter3.get("A"));
                        -        assertNull(getter3.get("B"));
                        -        assertEquals(1, (int) getter3.get("C"));
                        +            assertNull(getter2.get("A"));
                        +            assertEquals(2, (int) getter2.get("B"));
                        +            assertNull(getter2.get("C"));
                         
                        -        driver.process(topic1, "A", null);
                        -        driver.process(topic1, "B", null);
                        -        driver.flushState();
                        +            assertEquals(3, (int) getter3.get("A"));
                        +            assertNull(getter3.get("B"));
                        +            assertEquals(1, (int) getter3.get("C"));
                         
                        -        assertNull(getter2.get("A"));
                        -        assertNull(getter2.get("B"));
                        -        assertNull(getter2.get("C"));
                        +            driver.pipeInput(recordFactory.create(topic1, "A", null));
                        +            driver.pipeInput(recordFactory.create(topic1, "B", null));
                         
                        -        assertNull(getter3.get("A"));
                        -        assertNull(getter3.get("B"));
                        -        assertEquals(1, (int) getter3.get("C"));
                        +            assertNull(getter2.get("A"));
                        +            assertNull(getter2.get("B"));
                        +            assertNull(getter2.get("C"));
                        +
                        +            assertNull(getter3.get("A"));
                        +            assertNull(getter3.get("B"));
                        +            assertEquals(1, (int) getter3.get("C"));
                        +        }
                             }
                         
                             @Test
                        @@ -259,34 +257,34 @@ private void doTestNotSendingOldValue(final StreamsBuilder builder,
                                 builder.build().addProcessor("proc1", supplier, table1.name);
                                 builder.build().addProcessor("proc2", supplier, table2.name);
                         
                        -        driver.setUp(builder, stateDir, Serdes.String(), Serdes.Integer());
                        +        try (final TopologyTestDriver driver = new TopologyTestDriver(builder.build(), props)) {
                         
                        -        driver.process(topic1, "A", 1);
                        -        driver.process(topic1, "B", 1);
                        -        driver.process(topic1, "C", 1);
                        -        driver.flushState();
                        +            driver.pipeInput(recordFactory.create(topic1, "A", 1));
                        +            driver.pipeInput(recordFactory.create(topic1, "B", 1));
                        +            driver.pipeInput(recordFactory.create(topic1, "C", 1));
                         
                        -        final List> processors = supplier.capturedProcessors(2);
                        +            final List> processors = supplier.capturedProcessors(2);
                        +
                        +            processors.get(0).checkAndClearProcessResult("A:(1<-null)", "B:(1<-null)", "C:(1<-null)");
                        +            processors.get(1).checkAndClearProcessResult("A:(null<-null)", "B:(null<-null)", "C:(null<-null)");
                         
                        -        processors.get(0).checkAndClearProcessResult("A:(1<-null)", "B:(1<-null)", "C:(1<-null)");
                        -        processors.get(1).checkAndClearProcessResult("A:(null<-null)", "B:(null<-null)", "C:(null<-null)");
                        -
                        -        driver.process(topic1, "A", 2);
                        -        driver.process(topic1, "B", 2);
                        -        driver.flushState();
                        -        processors.get(0).checkAndClearProcessResult("A:(2<-null)", "B:(2<-null)");
                        -        processors.get(1).checkAndClearProcessResult("A:(2<-null)", "B:(2<-null)");
                        -
                        -        driver.process(topic1, "A", 3);
                        -        driver.flushState();
                        -        processors.get(0).checkAndClearProcessResult("A:(3<-null)");
                        -        processors.get(1).checkAndClearProcessResult("A:(null<-null)");
                        -
                        -        driver.process(topic1, "A", null);
                        -        driver.process(topic1, "B", null);
                        -        driver.flushState();
                        -        processors.get(0).checkAndClearProcessResult("A:(null<-null)", "B:(null<-null)");
                        -        processors.get(1).checkAndClearProcessResult("A:(null<-null)", "B:(null<-null)");
                        +            driver.pipeInput(recordFactory.create(topic1, "A", 2));
                        +            driver.pipeInput(recordFactory.create(topic1, "B", 2));
                        +
                        +            processors.get(0).checkAndClearProcessResult("A:(2<-null)", "B:(2<-null)");
                        +            processors.get(1).checkAndClearProcessResult("A:(2<-null)", "B:(2<-null)");
                        +
                        +            driver.pipeInput(recordFactory.create(topic1, "A", 3));
                        +
                        +            processors.get(0).checkAndClearProcessResult("A:(3<-null)");
                        +            processors.get(1).checkAndClearProcessResult("A:(null<-null)");
                        +
                        +            driver.pipeInput(recordFactory.create(topic1, "A", null));
                        +            driver.pipeInput(recordFactory.create(topic1, "B", null));
                        +
                        +            processors.get(0).checkAndClearProcessResult("A:(null<-null)", "B:(null<-null)");
                        +            processors.get(1).checkAndClearProcessResult("A:(null<-null)", "B:(null<-null)");
                        +        }
                             }
                         
                         
                        @@ -340,34 +338,34 @@ private void doTestSendingOldValue(final StreamsBuilder builder,
                                 topology.addProcessor("proc1", supplier, table1.name);
                                 topology.addProcessor("proc2", supplier, table2.name);
                         
                        -        driver.setUp(builder, stateDir, Serdes.String(), Serdes.Integer());
                        +        try (final TopologyTestDriver driver = new TopologyTestDriver(topology, props)) {
                         
                        -        driver.process(topic1, "A", 1);
                        -        driver.process(topic1, "B", 1);
                        -        driver.process(topic1, "C", 1);
                        -        driver.flushState();
                        +            driver.pipeInput(recordFactory.create(topic1, "A", 1));
                        +            driver.pipeInput(recordFactory.create(topic1, "B", 1));
                        +            driver.pipeInput(recordFactory.create(topic1, "C", 1));
                         
                        -        final List> processors = supplier.capturedProcessors(2);
                        +            final List> processors = supplier.capturedProcessors(2);
                         
                        -        processors.get(0).checkAndClearProcessResult("A:(1<-null)", "B:(1<-null)", "C:(1<-null)");
                        -        processors.get(1).checkEmptyAndClearProcessResult();
                        +            processors.get(0).checkAndClearProcessResult("A:(1<-null)", "B:(1<-null)", "C:(1<-null)");
                        +            processors.get(1).checkEmptyAndClearProcessResult();
                        +
                        +            driver.pipeInput(recordFactory.create(topic1, "A", 2));
                        +            driver.pipeInput(recordFactory.create(topic1, "B", 2));
                        +
                        +            processors.get(0).checkAndClearProcessResult("A:(2<-1)", "B:(2<-1)");
                        +            processors.get(1).checkAndClearProcessResult("A:(2<-null)", "B:(2<-null)");
                        +
                        +            driver.pipeInput(recordFactory.create(topic1, "A", 3));
                        +
                        +            processors.get(0).checkAndClearProcessResult("A:(3<-2)");
                        +            processors.get(1).checkAndClearProcessResult("A:(null<-2)");
                        +
                        +            driver.pipeInput(recordFactory.create(topic1, "A", null));
                        +            driver.pipeInput(recordFactory.create(topic1, "B", null));
                         
                        -        driver.process(topic1, "A", 2);
                        -        driver.process(topic1, "B", 2);
                        -        driver.flushState();
                        -        processors.get(0).checkAndClearProcessResult("A:(2<-1)", "B:(2<-1)");
                        -        processors.get(1).checkAndClearProcessResult("A:(2<-null)", "B:(2<-null)");
                        -
                        -        driver.process(topic1, "A", 3);
                        -        driver.flushState();
                        -        processors.get(0).checkAndClearProcessResult("A:(3<-2)");
                        -        processors.get(1).checkAndClearProcessResult("A:(null<-2)");
                        -
                        -        driver.process(topic1, "A", null);
                        -        driver.process(topic1, "B", null);
                        -        driver.flushState();
                        -        processors.get(0).checkAndClearProcessResult("A:(null<-3)", "B:(null<-2)");
                        -        processors.get(1).checkAndClearProcessResult("B:(null<-2)");
                        +            processors.get(0).checkAndClearProcessResult("A:(null<-3)", "B:(null<-2)");
                        +            processors.get(1).checkAndClearProcessResult("B:(null<-2)");
                        +        }
                             }
                         
                             @Test
                        @@ -418,12 +416,13 @@ private void doTestSkipNullOnMaterialization(final StreamsBuilder builder,
                                 topology.addProcessor("proc1", supplier, table1.name);
                                 topology.addProcessor("proc2", supplier, table2.name);
                         
                        -        driver.setUp(builder, stateDir, stringSerde, stringSerde);
                        +        final ConsumerRecordFactory stringRecordFactory = new ConsumerRecordFactory<>(new StringSerializer(), new StringSerializer());
                        +        try (final TopologyTestDriver driver = new TopologyTestDriver(topology, props)) {
                         
                        -        driver.process(topic1, "A", "reject");
                        -        driver.process(topic1, "B", "reject");
                        -        driver.process(topic1, "C", "reject");
                        -        driver.flushState();
                        +            driver.pipeInput(stringRecordFactory.create(topic1, "A", "reject"));
                        +            driver.pipeInput(stringRecordFactory.create(topic1, "B", "reject"));
                        +            driver.pipeInput(stringRecordFactory.create(topic1, "C", "reject"));
                        +        }
                         
                                 final List> processors = supplier.capturedProcessors(2);
                                 processors.get(0).checkAndClearProcessResult("A:(reject<-null)", "B:(reject<-null)", "C:(reject<-null)");
                        @@ -437,7 +436,7 @@ public void testSkipNullOnMaterialization() {
                         
                                 String topic1 = "topic1";
                         
                        -        final Consumed consumed = Consumed.with(stringSerde, stringSerde);
                        +        final Consumed consumed = Consumed.with(Serdes.String(), Serdes.String());
                                 KTableImpl table1 =
                                     (KTableImpl) builder.table(topic1, consumed);
                                 KTableImpl table2 = (KTableImpl) table1.filter(
                        @@ -459,7 +458,7 @@ public void testQueryableSkipNullOnMaterialization() {
                         
                                 String topic1 = "topic1";
                         
                        -        final Consumed consumed = Consumed.with(stringSerde, stringSerde);
                        +        final Consumed consumed = Consumed.with(Serdes.String(), Serdes.String());
                                 KTableImpl table1 =
                                     (KTableImpl) builder.table(topic1, consumed);
                                 KTableImpl table2 = (KTableImpl) table1.filter(
                        diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KTableImplTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KTableImplTest.java
                        index 0b9c1ab22d724..016cde2f1fba3 100644
                        --- a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KTableImplTest.java
                        +++ b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KTableImplTest.java
                        @@ -16,12 +16,17 @@
                          */
                         package org.apache.kafka.streams.kstream.internals;
                         
                        -import org.apache.kafka.common.serialization.Serde;
                         import org.apache.kafka.common.serialization.Serdes;
                        +import org.apache.kafka.common.serialization.StringSerializer;
                         import org.apache.kafka.common.utils.Bytes;
                         import org.apache.kafka.common.utils.Utils;
                        -import org.apache.kafka.streams.kstream.Consumed;
                         import org.apache.kafka.streams.StreamsBuilder;
                        +import org.apache.kafka.streams.Topology;
                        +import org.apache.kafka.streams.TopologyDescription;
                        +import org.apache.kafka.streams.TopologyTestDriver;
                        +import org.apache.kafka.streams.TopologyTestDriverWrapper;
                        +import org.apache.kafka.streams.TopologyWrapper;
                        +import org.apache.kafka.streams.kstream.Consumed;
                         import org.apache.kafka.streams.kstream.KTable;
                         import org.apache.kafka.streams.kstream.Materialized;
                         import org.apache.kafka.streams.kstream.Predicate;
                        @@ -30,10 +35,11 @@
                         import org.apache.kafka.streams.kstream.ValueMapper;
                         import org.apache.kafka.streams.kstream.ValueMapperWithKey;
                         import org.apache.kafka.streams.kstream.ValueTransformerWithKeySupplier;
                        +import org.apache.kafka.streams.processor.internals.InternalTopologyBuilder;
                         import org.apache.kafka.streams.processor.internals.SinkNode;
                         import org.apache.kafka.streams.processor.internals.SourceNode;
                         import org.apache.kafka.streams.state.KeyValueStore;
                        -import org.apache.kafka.test.KStreamTestDriver;
                        +import org.apache.kafka.streams.test.ConsumerRecordFactory;
                         import org.apache.kafka.test.MockAggregator;
                         import org.apache.kafka.test.MockInitializer;
                         import org.apache.kafka.test.MockMapper;
                        @@ -41,36 +47,31 @@
                         import org.apache.kafka.test.MockProcessorSupplier;
                         import org.apache.kafka.test.MockReducer;
                         import org.apache.kafka.test.MockValueJoiner;
                        -import org.apache.kafka.test.TestUtils;
                        +import org.apache.kafka.test.StreamsTestUtils;
                         import org.junit.Before;
                        -import org.junit.Rule;
                         import org.junit.Test;
                         
                        -import java.io.File;
                         import java.lang.reflect.Field;
                         import java.util.List;
                        +import java.util.Properties;
                         
                         import static org.easymock.EasyMock.mock;
                         import static org.junit.Assert.assertEquals;
                         import static org.junit.Assert.assertNotNull;
                         import static org.junit.Assert.assertNull;
                        -import static org.junit.Assert.assertTrue;
                         
                         public class KTableImplTest {
                         
                        -    private final Serde stringSerde = Serdes.String();
                        -    private final Consumed consumed = Consumed.with(stringSerde, stringSerde);
                        -    private final Produced produced = Produced.with(stringSerde, stringSerde);
                        +    private final Consumed consumed = Consumed.with(Serdes.String(), Serdes.String());
                        +    private final Produced produced = Produced.with(Serdes.String(), Serdes.String());
                        +    private final Properties props = StreamsTestUtils.topologyTestConfig(Serdes.String(), Serdes.String());
                        +    private final ConsumerRecordFactory recordFactory = new ConsumerRecordFactory<>(new StringSerializer(), new StringSerializer());
                         
                        -    @Rule
                        -    public final KStreamTestDriver driver = new KStreamTestDriver();
                        -    private File stateDir = null;
                             private StreamsBuilder builder;
                             private KTable table;
                         
                             @Before
                             public void setUp() {
                        -        stateDir = TestUtils.tempDirectory("kafka-test");
                                 builder = new StreamsBuilder();
                                 table = builder.table("test");
                             }
                        @@ -110,17 +111,12 @@ public boolean test(String key, Integer value) {
                         
                                 table4.toStream().process(supplier);
                         
                        -        driver.setUp(builder, stateDir);
                        -
                        -        driver.process(topic1, "A", "01");
                        -        driver.flushState();
                        -        driver.process(topic1, "B", "02");
                        -        driver.flushState();
                        -        driver.process(topic1, "C", "03");
                        -        driver.flushState();
                        -        driver.process(topic1, "D", "04");
                        -        driver.flushState();
                        -        driver.flushState();
                        +        try (final TopologyTestDriver driver = new TopologyTestDriver(builder.build(), props)) {
                        +            driver.pipeInput(recordFactory.create(topic1, "A", "01"));
                        +            driver.pipeInput(recordFactory.create(topic1, "B", "02"));
                        +            driver.pipeInput(recordFactory.create(topic1, "C", "03"));
                        +            driver.pipeInput(recordFactory.create(topic1, "D", "04"));
                        +        }
                         
                                 final List> processors = supplier.capturedProcessors(4);
                                 assertEquals(Utils.mkList("A:01", "B:02", "C:03", "D:04"), processors.get(0).processed);
                        @@ -156,104 +152,109 @@ public boolean test(String key, Integer value) {
                                 table1.toStream().to(topic2, produced);
                                 final KTableImpl table4 = (KTableImpl) builder.table(topic2, consumed);
                         
                        +        final Topology topology = builder.build();
                        +
                                 final KTableValueGetterSupplier getterSupplier1 = table1.valueGetterSupplier();
                                 final KTableValueGetterSupplier getterSupplier2 = table2.valueGetterSupplier();
                                 final KTableValueGetterSupplier getterSupplier3 = table3.valueGetterSupplier();
                                 final KTableValueGetterSupplier getterSupplier4 = table4.valueGetterSupplier();
                         
                        -        driver.setUp(builder, stateDir, null, null);
                        +        final InternalTopologyBuilder topologyBuilder = TopologyWrapper.getInternalTopologyBuilder(topology);
                        +        topologyBuilder.connectProcessorAndStateStores(table1.name, getterSupplier1.storeNames());
                        +        topologyBuilder.connectProcessorAndStateStores(table2.name, getterSupplier2.storeNames());
                        +        topologyBuilder.connectProcessorAndStateStores(table3.name, getterSupplier3.storeNames());
                        +        topologyBuilder.connectProcessorAndStateStores(table4.name, getterSupplier4.storeNames());
                        +
                        +        try (final TopologyTestDriverWrapper driver = new TopologyTestDriverWrapper(topology, props)) {
                         
                        -        // two state store should be created
                        -        assertEquals(2, driver.allStateStores().size());
                        +            assertEquals(2, driver.getAllStateStores().size());
                         
                        -        final KTableValueGetter getter1 = getterSupplier1.get();
                        -        getter1.init(driver.context());
                        -        final KTableValueGetter getter2 = getterSupplier2.get();
                        -        getter2.init(driver.context());
                        -        final KTableValueGetter getter3 = getterSupplier3.get();
                        -        getter3.init(driver.context());
                        -        final KTableValueGetter getter4 = getterSupplier4.get();
                        -        getter4.init(driver.context());
                        +            final KTableValueGetter getter1 = getterSupplier1.get();
                        +            final KTableValueGetter getter2 = getterSupplier2.get();
                        +            final KTableValueGetter getter3 = getterSupplier3.get();
                        +            final KTableValueGetter getter4 = getterSupplier4.get();
                         
                        -        driver.process(topic1, "A", "01");
                        -        driver.process(topic1, "B", "01");
                        -        driver.process(topic1, "C", "01");
                        -        driver.flushState();
                        +            getter1.init(driver.setCurrentNodeForProcessorContext(table1.name));
                        +            getter2.init(driver.setCurrentNodeForProcessorContext(table2.name));
                        +            getter3.init(driver.setCurrentNodeForProcessorContext(table3.name));
                        +            getter4.init(driver.setCurrentNodeForProcessorContext(table4.name));
                         
                        -        assertEquals("01", getter1.get("A"));
                        -        assertEquals("01", getter1.get("B"));
                        -        assertEquals("01", getter1.get("C"));
                        +            driver.pipeInput(recordFactory.create(topic1, "A", "01"));
                        +            driver.pipeInput(recordFactory.create(topic1, "B", "01"));
                        +            driver.pipeInput(recordFactory.create(topic1, "C", "01"));
                         
                        -        assertEquals(new Integer(1), getter2.get("A"));
                        -        assertEquals(new Integer(1), getter2.get("B"));
                        -        assertEquals(new Integer(1), getter2.get("C"));
                        +            assertEquals("01", getter1.get("A"));
                        +            assertEquals("01", getter1.get("B"));
                        +            assertEquals("01", getter1.get("C"));
                         
                        -        assertNull(getter3.get("A"));
                        -        assertNull(getter3.get("B"));
                        -        assertNull(getter3.get("C"));
                        +            assertEquals(new Integer(1), getter2.get("A"));
                        +            assertEquals(new Integer(1), getter2.get("B"));
                        +            assertEquals(new Integer(1), getter2.get("C"));
                         
                        -        assertEquals("01", getter4.get("A"));
                        -        assertEquals("01", getter4.get("B"));
                        -        assertEquals("01", getter4.get("C"));
                        +            assertNull(getter3.get("A"));
                        +            assertNull(getter3.get("B"));
                        +            assertNull(getter3.get("C"));
                         
                        -        driver.process(topic1, "A", "02");
                        -        driver.process(topic1, "B", "02");
                        -        driver.flushState();
                        +            assertEquals("01", getter4.get("A"));
                        +            assertEquals("01", getter4.get("B"));
                        +            assertEquals("01", getter4.get("C"));
                         
                        -        assertEquals("02", getter1.get("A"));
                        -        assertEquals("02", getter1.get("B"));
                        -        assertEquals("01", getter1.get("C"));
                        +            driver.pipeInput(recordFactory.create(topic1, "A", "02"));
                        +            driver.pipeInput(recordFactory.create(topic1, "B", "02"));
                         
                        -        assertEquals(new Integer(2), getter2.get("A"));
                        -        assertEquals(new Integer(2), getter2.get("B"));
                        -        assertEquals(new Integer(1), getter2.get("C"));
                        +            assertEquals("02", getter1.get("A"));
                        +            assertEquals("02", getter1.get("B"));
                        +            assertEquals("01", getter1.get("C"));
                         
                        -        assertEquals(new Integer(2), getter3.get("A"));
                        -        assertEquals(new Integer(2), getter3.get("B"));
                        -        assertNull(getter3.get("C"));
                        +            assertEquals(new Integer(2), getter2.get("A"));
                        +            assertEquals(new Integer(2), getter2.get("B"));
                        +            assertEquals(new Integer(1), getter2.get("C"));
                         
                        -        assertEquals("02", getter4.get("A"));
                        -        assertEquals("02", getter4.get("B"));
                        -        assertEquals("01", getter4.get("C"));
                        +            assertEquals(new Integer(2), getter3.get("A"));
                        +            assertEquals(new Integer(2), getter3.get("B"));
                        +            assertNull(getter3.get("C"));
                         
                        -        driver.process(topic1, "A", "03");
                        -        driver.flushState();
                        +            assertEquals("02", getter4.get("A"));
                        +            assertEquals("02", getter4.get("B"));
                        +            assertEquals("01", getter4.get("C"));
                         
                        -        assertEquals("03", getter1.get("A"));
                        -        assertEquals("02", getter1.get("B"));
                        -        assertEquals("01", getter1.get("C"));
                        +            driver.pipeInput(recordFactory.create(topic1, "A", "03"));
                         
                        -        assertEquals(new Integer(3), getter2.get("A"));
                        -        assertEquals(new Integer(2), getter2.get("B"));
                        -        assertEquals(new Integer(1), getter2.get("C"));
                        +            assertEquals("03", getter1.get("A"));
                        +            assertEquals("02", getter1.get("B"));
                        +            assertEquals("01", getter1.get("C"));
                         
                        -        assertNull(getter3.get("A"));
                        -        assertEquals(new Integer(2), getter3.get("B"));
                        -        assertNull(getter3.get("C"));
                        +            assertEquals(new Integer(3), getter2.get("A"));
                        +            assertEquals(new Integer(2), getter2.get("B"));
                        +            assertEquals(new Integer(1), getter2.get("C"));
                         
                        -        assertEquals("03", getter4.get("A"));
                        -        assertEquals("02", getter4.get("B"));
                        -        assertEquals("01", getter4.get("C"));
                        +            assertNull(getter3.get("A"));
                        +            assertEquals(new Integer(2), getter3.get("B"));
                        +            assertNull(getter3.get("C"));
                         
                        -        driver.process(topic1, "A", null);
                        -        driver.flushState();
                        +            assertEquals("03", getter4.get("A"));
                        +            assertEquals("02", getter4.get("B"));
                        +            assertEquals("01", getter4.get("C"));
                         
                        -        assertNull(getter1.get("A"));
                        -        assertEquals("02", getter1.get("B"));
                        -        assertEquals("01", getter1.get("C"));
                        +            driver.pipeInput(recordFactory.create(topic1, "A", (String) null));
                         
                        +            assertNull(getter1.get("A"));
                        +            assertEquals("02", getter1.get("B"));
                        +            assertEquals("01", getter1.get("C"));
                         
                        -        assertNull(getter2.get("A"));
                        -        assertEquals(new Integer(2), getter2.get("B"));
                        -        assertEquals(new Integer(1), getter2.get("C"));
                         
                        -        assertNull(getter3.get("A"));
                        -        assertEquals(new Integer(2), getter3.get("B"));
                        -        assertNull(getter3.get("C"));
                        +            assertNull(getter2.get("A"));
                        +            assertEquals(new Integer(2), getter2.get("B"));
                        +            assertEquals(new Integer(1), getter2.get("C"));
                         
                        -        assertNull(getter4.get("A"));
                        -        assertEquals("02", getter4.get("B"));
                        -        assertEquals("01", getter4.get("C"));
                        +            assertNull(getter3.get("A"));
                        +            assertEquals(new Integer(2), getter3.get("B"));
                        +            assertNull(getter3.get("C"));
                        +
                        +            assertNull(getter4.get("A"));
                        +            assertEquals("02", getter4.get("B"));
                        +            assertEquals("01", getter4.get("C"));
                        +        }
                             }
                         
                             @Test
                        @@ -282,11 +283,9 @@ public boolean test(String key, Integer value) {
                                             }
                                         });
                         
                        -        driver.setUp(builder, stateDir, null, null);
                        -        driver.setTime(0L);
                        -
                        -        // two state stores should be created
                        -        assertEquals(2, driver.allStateStores().size());
                        +        try (final TopologyTestDriver driver = new TopologyTestDriver(builder.build(), props)) {
                        +            assertEquals(2, driver.getAllStateStores().size());
                        +        }
                             }
                         
                             @Test
                        @@ -323,15 +322,25 @@ public String apply(String v1, Integer v2) {
                                             }
                                         });
                         
                        -        driver.setUp(builder, stateDir, null, null);
                        -        driver.setTime(0L);
                        +        try (final TopologyTestDriver driver = new TopologyTestDriver(builder.build(), props)) {
                        +            assertEquals(2, driver.getAllStateStores().size());
                        +        }
                        +    }
                         
                        -        // two state store should be created
                        -        assertEquals(2, driver.allStateStores().size());
                        +    private void assertTopologyContainsProcessor(final Topology topology, final String processorName) {
                        +        for (final TopologyDescription.Subtopology subtopology: topology.describe().subtopologies()) {
                        +            for (final TopologyDescription.Node node: subtopology.nodes()) {
                        +                if (node.name().equals(processorName)) {
                        +                    return;
                        +                }
                        +            }
                        +        }
                        +        throw new AssertionError("No processor named '" + processorName + "'"
                        +                + "found in the provided Topology:\n" + topology.describe());
                             }
                         
                             @Test
                        -    public void testRepartition() throws NoSuchFieldException, IllegalAccessException {
                        +    public void shouldCreateSourceAndSinkNodesForRepartitioningTopic() throws NoSuchFieldException, IllegalAccessException {
                                 final String topic1 = "topic1";
                                 final String storeName1 = "storeName1";
                         
                        @@ -341,8 +350,8 @@ public void testRepartition() throws NoSuchFieldException, IllegalAccessExceptio
                                         (KTableImpl) builder.table(topic1,
                                                                                            consumed,
                                                                                            Materialized.>as(storeName1)
                        -                                                                           .withKeySerde(stringSerde)
                        -                                                                           .withValueSerde(stringSerde)
                        +                                                                           .withKeySerde(Serdes.String())
                        +                                                                           .withValueSerde(Serdes.String())
                                         );
                         
                                 table1.groupBy(MockMapper.noOpKeyValueMapper())
                        @@ -352,27 +361,26 @@ public void testRepartition() throws NoSuchFieldException, IllegalAccessExceptio
                                 table1.groupBy(MockMapper.noOpKeyValueMapper())
                                     .reduce(MockReducer.STRING_ADDER, MockReducer.STRING_REMOVER, Materialized.>as("mock-result2"));
                         
                        -        driver.setUp(builder, stateDir, stringSerde, stringSerde);
                        -        driver.setTime(0L);
                        +        final Topology topology = builder.build();
                        +        try (final TopologyTestDriverWrapper driver = new TopologyTestDriverWrapper(topology, props)) {
                         
                        -        // three state store should be created, one for source, one for aggregate and one for reduce
                        -        assertEquals(3, driver.allStateStores().size());
                        +            assertEquals(3, driver.getAllStateStores().size());
                         
                        -        // contains the corresponding repartition source / sink nodes
                        -        assertTrue(driver.allProcessorNames().contains("KSTREAM-SINK-0000000003"));
                        -        assertTrue(driver.allProcessorNames().contains("KSTREAM-SOURCE-0000000004"));
                        -        assertTrue(driver.allProcessorNames().contains("KSTREAM-SINK-0000000007"));
                        -        assertTrue(driver.allProcessorNames().contains("KSTREAM-SOURCE-0000000008"));
                        +            assertTopologyContainsProcessor(topology, "KSTREAM-SINK-0000000003");
                        +            assertTopologyContainsProcessor(topology, "KSTREAM-SOURCE-0000000004");
                        +            assertTopologyContainsProcessor(topology, "KSTREAM-SINK-0000000007");
                        +            assertTopologyContainsProcessor(topology, "KSTREAM-SOURCE-0000000008");
                         
                        -        Field valSerializerField  = ((SinkNode) driver.processor("KSTREAM-SINK-0000000003")).getClass().getDeclaredField("valSerializer");
                        -        Field valDeserializerField  = ((SourceNode) driver.processor("KSTREAM-SOURCE-0000000004")).getClass().getDeclaredField("valDeserializer");
                        -        valSerializerField.setAccessible(true);
                        -        valDeserializerField.setAccessible(true);
                        +            Field valSerializerField = ((SinkNode) driver.getProcessor("KSTREAM-SINK-0000000003")).getClass().getDeclaredField("valSerializer");
                        +            Field valDeserializerField = ((SourceNode) driver.getProcessor("KSTREAM-SOURCE-0000000004")).getClass().getDeclaredField("valDeserializer");
                        +            valSerializerField.setAccessible(true);
                        +            valDeserializerField.setAccessible(true);
                         
                        -        assertNotNull(((ChangedSerializer) valSerializerField.get(driver.processor("KSTREAM-SINK-0000000003"))).inner());
                        -        assertNotNull(((ChangedDeserializer) valDeserializerField.get(driver.processor("KSTREAM-SOURCE-0000000004"))).inner());
                        -        assertNotNull(((ChangedSerializer) valSerializerField.get(driver.processor("KSTREAM-SINK-0000000007"))).inner());
                        -        assertNotNull(((ChangedDeserializer) valDeserializerField.get(driver.processor("KSTREAM-SOURCE-0000000008"))).inner());
                        +            assertNotNull(((ChangedSerializer) valSerializerField.get(driver.getProcessor("KSTREAM-SINK-0000000003"))).inner());
                        +            assertNotNull(((ChangedDeserializer) valDeserializerField.get(driver.getProcessor("KSTREAM-SOURCE-0000000004"))).inner());
                        +            assertNotNull(((ChangedSerializer) valSerializerField.get(driver.getProcessor("KSTREAM-SINK-0000000007"))).inner());
                        +            assertNotNull(((ChangedDeserializer) valDeserializerField.get(driver.getProcessor("KSTREAM-SOURCE-0000000008"))).inner());
                        +        }
                             }
                         
                             @Test(expected = NullPointerException.class)
                        diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KTableMapValuesTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KTableMapValuesTest.java
                        index a54e43e92c36a..a01d5cb215a02 100644
                        --- a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KTableMapValuesTest.java
                        +++ b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KTableMapValuesTest.java
                        @@ -16,27 +16,30 @@
                          */
                         package org.apache.kafka.streams.kstream.internals;
                         
                        -import org.apache.kafka.common.serialization.Serde;
                         import org.apache.kafka.common.serialization.Serdes;
                        +import org.apache.kafka.common.serialization.StringSerializer;
                         import org.apache.kafka.common.utils.Bytes;
                         import org.apache.kafka.common.utils.Utils;
                        -import org.apache.kafka.streams.kstream.Consumed;
                         import org.apache.kafka.streams.StreamsBuilder;
                        +import org.apache.kafka.streams.Topology;
                        +import org.apache.kafka.streams.TopologyTestDriver;
                        +import org.apache.kafka.streams.TopologyTestDriverWrapper;
                        +import org.apache.kafka.streams.TopologyWrapper;
                        +import org.apache.kafka.streams.kstream.Consumed;
                         import org.apache.kafka.streams.kstream.KTable;
                         import org.apache.kafka.streams.kstream.Materialized;
                         import org.apache.kafka.streams.kstream.Predicate;
                         import org.apache.kafka.streams.kstream.Produced;
                         import org.apache.kafka.streams.kstream.ValueMapper;
                        +import org.apache.kafka.streams.processor.internals.InternalTopologyBuilder;
                         import org.apache.kafka.streams.state.KeyValueStore;
                        -import org.apache.kafka.test.KStreamTestDriver;
                        +import org.apache.kafka.streams.test.ConsumerRecordFactory;
                         import org.apache.kafka.test.MockProcessor;
                         import org.apache.kafka.test.MockProcessorSupplier;
                        -import org.apache.kafka.test.TestUtils;
                        -import org.junit.Before;
                        -import org.junit.Rule;
                        +import org.apache.kafka.test.StreamsTestUtils;
                         import org.junit.Test;
                         
                        -import java.io.File;
                        +import java.util.Properties;
                         
                         import static org.junit.Assert.assertEquals;
                         import static org.junit.Assert.assertFalse;
                        @@ -45,27 +48,19 @@
                         
                         public class KTableMapValuesTest {
                         
                        -    private final Serde stringSerde = Serdes.String();
                        -    private final Consumed consumed = Consumed.with(stringSerde, stringSerde);
                        -    private final Produced produced = Produced.with(stringSerde, stringSerde);
                        -    @Rule
                        -    public final KStreamTestDriver driver = new KStreamTestDriver();
                        -    private File stateDir = null;
                        -
                        -    @Before
                        -    public void setUp() {
                        -        stateDir = TestUtils.tempDirectory("kafka-test");
                        -    }
                        +    private final Consumed consumed = Consumed.with(Serdes.String(), Serdes.String());
                        +    private final Produced produced = Produced.with(Serdes.String(), Serdes.String());
                        +    private final ConsumerRecordFactory recordFactory = new ConsumerRecordFactory<>(new StringSerializer(), new StringSerializer());
                        +    private final Properties props = StreamsTestUtils.topologyTestConfig(Serdes.String(), Serdes.String());
                         
                             private void doTestKTable(final StreamsBuilder builder, final String topic1, final MockProcessorSupplier supplier) {
                        -        driver.setUp(builder, stateDir, Serdes.String(), Serdes.String());
                        -
                        -        driver.process(topic1, "A", "1");
                        -        driver.process(topic1, "B", "2");
                        -        driver.process(topic1, "C", "3");
                        -        driver.process(topic1, "D", "4");
                        -        driver.flushState();
                        -        assertEquals(Utils.mkList("A:1", "B:2", "C:3", "D:4"), supplier.theCapturedProcessor().processed);
                        +        try (final TopologyTestDriver driver = new TopologyTestDriver(builder.build(), props)) {
                        +            driver.pipeInput(recordFactory.create(topic1, "A", "1"));
                        +            driver.pipeInput(recordFactory.create(topic1, "B", "2"));
                        +            driver.pipeInput(recordFactory.create(topic1, "C", "3"));
                        +            driver.pipeInput(recordFactory.create(topic1, "D", "4"));
                        +            assertEquals(Utils.mkList("A:1", "B:2", "C:3", "D:4"), supplier.theCapturedProcessor().processed);
                        +        }
                             }
                         
                             @Test
                        @@ -114,99 +109,106 @@ private void doTestValueGetter(final StreamsBuilder builder,
                                                            final KTableImpl table2,
                                                            final KTableImpl table3,
                                                            final KTableImpl table4) {
                        +
                        +        final Topology topology = builder.build();
                        +
                                 final KTableValueGetterSupplier getterSupplier1 = table1.valueGetterSupplier();
                                 final KTableValueGetterSupplier getterSupplier2 = table2.valueGetterSupplier();
                                 final KTableValueGetterSupplier getterSupplier3 = table3.valueGetterSupplier();
                                 final KTableValueGetterSupplier getterSupplier4 = table4.valueGetterSupplier();
                         
                        -        driver.setUp(builder, stateDir, Serdes.String(), Serdes.String());
                        -        final KTableValueGetter getter1 = getterSupplier1.get();
                        -        getter1.init(driver.context());
                        -        final KTableValueGetter getter2 = getterSupplier2.get();
                        -        getter2.init(driver.context());
                        -        final KTableValueGetter getter3 = getterSupplier3.get();
                        -        getter3.init(driver.context());
                        -        final KTableValueGetter getter4 = getterSupplier4.get();
                        -        getter4.init(driver.context());
                        -
                        -        driver.process(topic1, "A", "01");
                        -        driver.process(topic1, "B", "01");
                        -        driver.process(topic1, "C", "01");
                        -        driver.flushState();
                        -
                        -        assertEquals("01", getter1.get("A"));
                        -        assertEquals("01", getter1.get("B"));
                        -        assertEquals("01", getter1.get("C"));
                        -
                        -        assertEquals(new Integer(1), getter2.get("A"));
                        -        assertEquals(new Integer(1), getter2.get("B"));
                        -        assertEquals(new Integer(1), getter2.get("C"));
                        -
                        -        assertNull(getter3.get("A"));
                        -        assertNull(getter3.get("B"));
                        -        assertNull(getter3.get("C"));
                        -
                        -        assertEquals("01", getter4.get("A"));
                        -        assertEquals("01", getter4.get("B"));
                        -        assertEquals("01", getter4.get("C"));
                        -
                        -        driver.process(topic1, "A", "02");
                        -        driver.process(topic1, "B", "02");
                        -        driver.flushState();
                        -
                        -        assertEquals("02", getter1.get("A"));
                        -        assertEquals("02", getter1.get("B"));
                        -        assertEquals("01", getter1.get("C"));
                        -
                        -        assertEquals(new Integer(2), getter2.get("A"));
                        -        assertEquals(new Integer(2), getter2.get("B"));
                        -        assertEquals(new Integer(1), getter2.get("C"));
                        -
                        -        assertEquals(new Integer(2), getter3.get("A"));
                        -        assertEquals(new Integer(2), getter3.get("B"));
                        -        assertNull(getter3.get("C"));
                        -
                        -        assertEquals("02", getter4.get("A"));
                        -        assertEquals("02", getter4.get("B"));
                        -        assertEquals("01", getter4.get("C"));
                        -
                        -        driver.process(topic1, "A", "03");
                        -        driver.flushState();
                        -
                        -        assertEquals("03", getter1.get("A"));
                        -        assertEquals("02", getter1.get("B"));
                        -        assertEquals("01", getter1.get("C"));
                        -
                        -        assertEquals(new Integer(3), getter2.get("A"));
                        -        assertEquals(new Integer(2), getter2.get("B"));
                        -        assertEquals(new Integer(1), getter2.get("C"));
                        -
                        -        assertNull(getter3.get("A"));
                        -        assertEquals(new Integer(2), getter3.get("B"));
                        -        assertNull(getter3.get("C"));
                        -
                        -        assertEquals("03", getter4.get("A"));
                        -        assertEquals("02", getter4.get("B"));
                        -        assertEquals("01", getter4.get("C"));
                        -
                        -        driver.process(topic1, "A", null);
                        -        driver.flushState();
                        -
                        -        assertNull(getter1.get("A"));
                        -        assertEquals("02", getter1.get("B"));
                        -        assertEquals("01", getter1.get("C"));
                        -
                        -        assertNull(getter2.get("A"));
                        -        assertEquals(new Integer(2), getter2.get("B"));
                        -        assertEquals(new Integer(1), getter2.get("C"));
                        -
                        -        assertNull(getter3.get("A"));
                        -        assertEquals(new Integer(2), getter3.get("B"));
                        -        assertNull(getter3.get("C"));
                        -
                        -        assertNull(getter4.get("A"));
                        -        assertEquals("02", getter4.get("B"));
                        -        assertEquals("01", getter4.get("C"));
                        +        final InternalTopologyBuilder topologyBuilder = TopologyWrapper.getInternalTopologyBuilder(topology);
                        +        topologyBuilder.connectProcessorAndStateStores(table1.name, getterSupplier1.storeNames());
                        +        topologyBuilder.connectProcessorAndStateStores(table2.name, getterSupplier2.storeNames());
                        +        topologyBuilder.connectProcessorAndStateStores(table3.name, getterSupplier3.storeNames());
                        +        topologyBuilder.connectProcessorAndStateStores(table4.name, getterSupplier4.storeNames());
                        +
                        +        try (final TopologyTestDriverWrapper driver = new TopologyTestDriverWrapper(builder.build(), props)) {
                        +            KTableValueGetter getter1 = getterSupplier1.get();
                        +            KTableValueGetter getter2 = getterSupplier2.get();
                        +            KTableValueGetter getter3 = getterSupplier3.get();
                        +            KTableValueGetter getter4 = getterSupplier4.get();
                        +
                        +            getter1.init(driver.setCurrentNodeForProcessorContext(table1.name));
                        +            getter2.init(driver.setCurrentNodeForProcessorContext(table2.name));
                        +            getter3.init(driver.setCurrentNodeForProcessorContext(table3.name));
                        +            getter4.init(driver.setCurrentNodeForProcessorContext(table4.name));
                        +
                        +            driver.pipeInput(recordFactory.create(topic1, "A", "01"));
                        +            driver.pipeInput(recordFactory.create(topic1, "B", "01"));
                        +            driver.pipeInput(recordFactory.create(topic1, "C", "01"));
                        +
                        +            assertEquals("01", getter1.get("A"));
                        +            assertEquals("01", getter1.get("B"));
                        +            assertEquals("01", getter1.get("C"));
                        +
                        +            assertEquals(new Integer(1), getter2.get("A"));
                        +            assertEquals(new Integer(1), getter2.get("B"));
                        +            assertEquals(new Integer(1), getter2.get("C"));
                        +
                        +            assertNull(getter3.get("A"));
                        +            assertNull(getter3.get("B"));
                        +            assertNull(getter3.get("C"));
                        +
                        +            assertEquals("01", getter4.get("A"));
                        +            assertEquals("01", getter4.get("B"));
                        +            assertEquals("01", getter4.get("C"));
                        +
                        +            driver.pipeInput(recordFactory.create(topic1, "A", "02"));
                        +            driver.pipeInput(recordFactory.create(topic1, "B", "02"));
                        +
                        +            assertEquals("02", getter1.get("A"));
                        +            assertEquals("02", getter1.get("B"));
                        +            assertEquals("01", getter1.get("C"));
                        +
                        +            assertEquals(new Integer(2), getter2.get("A"));
                        +            assertEquals(new Integer(2), getter2.get("B"));
                        +            assertEquals(new Integer(1), getter2.get("C"));
                        +
                        +            assertEquals(new Integer(2), getter3.get("A"));
                        +            assertEquals(new Integer(2), getter3.get("B"));
                        +            assertNull(getter3.get("C"));
                        +
                        +            assertEquals("02", getter4.get("A"));
                        +            assertEquals("02", getter4.get("B"));
                        +            assertEquals("01", getter4.get("C"));
                        +
                        +            driver.pipeInput(recordFactory.create(topic1, "A", "03"));
                        +
                        +            assertEquals("03", getter1.get("A"));
                        +            assertEquals("02", getter1.get("B"));
                        +            assertEquals("01", getter1.get("C"));
                        +
                        +            assertEquals(new Integer(3), getter2.get("A"));
                        +            assertEquals(new Integer(2), getter2.get("B"));
                        +            assertEquals(new Integer(1), getter2.get("C"));
                        +
                        +            assertNull(getter3.get("A"));
                        +            assertEquals(new Integer(2), getter3.get("B"));
                        +            assertNull(getter3.get("C"));
                        +
                        +            assertEquals("03", getter4.get("A"));
                        +            assertEquals("02", getter4.get("B"));
                        +            assertEquals("01", getter4.get("C"));
                        +
                        +            driver.pipeInput(recordFactory.create(topic1, "A", (String) null));
                        +
                        +            assertNull(getter1.get("A"));
                        +            assertEquals("02", getter1.get("B"));
                        +            assertEquals("01", getter1.get("C"));
                        +
                        +            assertNull(getter2.get("A"));
                        +            assertEquals(new Integer(2), getter2.get("B"));
                        +            assertEquals(new Integer(1), getter2.get("C"));
                        +
                        +            assertNull(getter3.get("A"));
                        +            assertEquals(new Integer(2), getter3.get("B"));
                        +            assertNull(getter3.get("C"));
                        +
                        +            assertNull(getter4.get("A"));
                        +            assertEquals("02", getter4.get("B"));
                        +            assertEquals("01", getter4.get("C"));
                        +        }
                             }
                         
                             @Test
                        @@ -244,6 +246,8 @@ public void testQueryableValueGetter() {
                         
                                 final String topic1 = "topic1";
                                 final String topic2 = "topic2";
                        +        final String storeName2 = "anyMapName";
                        +        final String storeName3 = "anyFilterName";
                         
                                 final KTableImpl table1 =
                                     (KTableImpl) builder.table(topic1, consumed);
                        @@ -253,14 +257,14 @@ public void testQueryableValueGetter() {
                                         public Integer apply(String value) {
                                             return new Integer(value);
                                         }
                        -            }, Materialized.>as("anyMapName").withValueSerde(Serdes.Integer()));
                        +            }, Materialized.>as(storeName2).withValueSerde(Serdes.Integer()));
                                 final KTableImpl table3 = (KTableImpl) table2.filter(
                                     new Predicate() {
                                         @Override
                                         public boolean test(String key, Integer value) {
                                             return (value % 2) == 0;
                                         }
                        -            }, Materialized.>as("anyFilterName").withValueSerde(Serdes.Integer()));
                        +            }, Materialized.>as(storeName3).withValueSerde(Serdes.Integer()));
                                 table1.toStream().to(topic2, produced);
                                 final KTableImpl table4 = (KTableImpl) builder.table(topic2, consumed);
                         
                        @@ -285,37 +289,34 @@ public Integer apply(String value) {
                         
                                 final MockProcessorSupplier supplier = new MockProcessorSupplier<>();
                         
                        -        builder.build().addProcessor("proc", supplier, table2.name);
                        +        final Topology topology = builder.build().addProcessor("proc", supplier, table2.name);
                         
                        -        driver.setUp(builder, stateDir);
                        +        try (final TopologyTestDriver driver = new TopologyTestDriver(topology, props)) {
                         
                        -        final MockProcessor proc = supplier.theCapturedProcessor();
                        +            final MockProcessor proc = supplier.theCapturedProcessor();
                         
                        -        assertFalse(table1.sendingOldValueEnabled());
                        -        assertFalse(table2.sendingOldValueEnabled());
                        +            assertFalse(table1.sendingOldValueEnabled());
                        +            assertFalse(table2.sendingOldValueEnabled());
                         
                        -        driver.process(topic1, "A", "01");
                        -        driver.process(topic1, "B", "01");
                        -        driver.process(topic1, "C", "01");
                        -        driver.flushState();
                        +            driver.pipeInput(recordFactory.create(topic1, "A", "01"));
                        +            driver.pipeInput(recordFactory.create(topic1, "B", "01"));
                        +            driver.pipeInput(recordFactory.create(topic1, "C", "01"));
                         
                        -        proc.checkAndClearProcessResult("A:(1<-null)", "B:(1<-null)", "C:(1<-null)");
                        +            proc.checkAndClearProcessResult("A:(1<-null)", "B:(1<-null)", "C:(1<-null)");
                         
                        -        driver.process(topic1, "A", "02");
                        -        driver.process(topic1, "B", "02");
                        -        driver.flushState();
                        +            driver.pipeInput(recordFactory.create(topic1, "A", "02"));
                        +            driver.pipeInput(recordFactory.create(topic1, "B", "02"));
                         
                        -        proc.checkAndClearProcessResult("A:(2<-null)", "B:(2<-null)");
                        +            proc.checkAndClearProcessResult("A:(2<-null)", "B:(2<-null)");
                         
                        -        driver.process(topic1, "A", "03");
                        -        driver.flushState();
                        +            driver.pipeInput(recordFactory.create(topic1, "A", "03"));
                         
                        -        proc.checkAndClearProcessResult("A:(3<-null)");
                        +            proc.checkAndClearProcessResult("A:(3<-null)");
                         
                        -        driver.process(topic1, "A", null);
                        -        driver.flushState();
                        +            driver.pipeInput(recordFactory.create(topic1, "A", (String) null));
                         
                        -        proc.checkAndClearProcessResult("A:(null<-null)");
                        +            proc.checkAndClearProcessResult("A:(null<-null)");
                        +        }
                             }
                         
                             @Test
                        @@ -340,34 +341,31 @@ public Integer apply(String value) {
                         
                                 builder.build().addProcessor("proc", supplier, table2.name);
                         
                        -        driver.setUp(builder, stateDir);
                        +        try (final TopologyTestDriver driver = new TopologyTestDriver(builder.build(), props)) {
                         
                        -        final MockProcessor proc = supplier.theCapturedProcessor();
                        +            final MockProcessor proc = supplier.theCapturedProcessor();
                         
                        -        assertTrue(table1.sendingOldValueEnabled());
                        -        assertTrue(table2.sendingOldValueEnabled());
                        +            assertTrue(table1.sendingOldValueEnabled());
                        +            assertTrue(table2.sendingOldValueEnabled());
                         
                        -        driver.process(topic1, "A", "01");
                        -        driver.process(topic1, "B", "01");
                        -        driver.process(topic1, "C", "01");
                        -        driver.flushState();
                        +            driver.pipeInput(recordFactory.create(topic1, "A", "01"));
                        +            driver.pipeInput(recordFactory.create(topic1, "B", "01"));
                        +            driver.pipeInput(recordFactory.create(topic1, "C", "01"));
                         
                        -        proc.checkAndClearProcessResult("A:(1<-null)", "B:(1<-null)", "C:(1<-null)");
                        +            proc.checkAndClearProcessResult("A:(1<-null)", "B:(1<-null)", "C:(1<-null)");
                         
                        -        driver.process(topic1, "A", "02");
                        -        driver.process(topic1, "B", "02");
                        -        driver.flushState();
                        +            driver.pipeInput(recordFactory.create(topic1, "A", "02"));
                        +            driver.pipeInput(recordFactory.create(topic1, "B", "02"));
                         
                        -        proc.checkAndClearProcessResult("A:(2<-1)", "B:(2<-1)");
                        +            proc.checkAndClearProcessResult("A:(2<-1)", "B:(2<-1)");
                         
                        -        driver.process(topic1, "A", "03");
                        -        driver.flushState();
                        +            driver.pipeInput(recordFactory.create(topic1, "A", "03"));
                         
                        -        proc.checkAndClearProcessResult("A:(3<-2)");
                        +            proc.checkAndClearProcessResult("A:(3<-2)");
                         
                        -        driver.process(topic1, "A", null);
                        -        driver.flushState();
                        +            driver.pipeInput(recordFactory.create(topic1, "A", (String) null));
                         
                        -        proc.checkAndClearProcessResult("A:(null<-3)");
                        +            proc.checkAndClearProcessResult("A:(null<-3)");
                        +        }
                             }
                         }
                        diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KTableSourceTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KTableSourceTest.java
                        index 504d8414aaea6..80a60ab2f203d 100644
                        --- a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KTableSourceTest.java
                        +++ b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KTableSourceTest.java
                        @@ -16,22 +16,26 @@
                          */
                         package org.apache.kafka.streams.kstream.internals;
                         
                        -import org.apache.kafka.common.serialization.Serde;
                        +import org.apache.kafka.common.serialization.IntegerSerializer;
                         import org.apache.kafka.common.serialization.Serdes;
                        +import org.apache.kafka.common.serialization.StringSerializer;
                         import org.apache.kafka.common.utils.Utils;
                        -import org.apache.kafka.streams.kstream.Consumed;
                         import org.apache.kafka.streams.StreamsBuilder;
                        +import org.apache.kafka.streams.Topology;
                        +import org.apache.kafka.streams.TopologyTestDriver;
                        +import org.apache.kafka.streams.TopologyTestDriverWrapper;
                        +import org.apache.kafka.streams.TopologyWrapper;
                        +import org.apache.kafka.streams.kstream.Consumed;
                         import org.apache.kafka.streams.kstream.KTable;
                        +import org.apache.kafka.streams.processor.internals.InternalTopologyBuilder;
                         import org.apache.kafka.streams.processor.internals.testutil.LogCaptureAppender;
                        -import org.apache.kafka.test.KStreamTestDriver;
                        +import org.apache.kafka.streams.test.ConsumerRecordFactory;
                         import org.apache.kafka.test.MockProcessor;
                         import org.apache.kafka.test.MockProcessorSupplier;
                        -import org.apache.kafka.test.TestUtils;
                        -import org.junit.Before;
                        -import org.junit.Rule;
                        +import org.apache.kafka.test.StreamsTestUtils;
                         import org.junit.Test;
                         
                        -import java.io.File;
                        +import java.util.Properties;
                         
                         import static org.apache.kafka.test.StreamsTestUtils.getMetricByName;
                         import static org.hamcrest.CoreMatchers.hasItem;
                        @@ -42,17 +46,9 @@
                         
                         public class KTableSourceTest {
                         
                        -    final private Serde stringSerde = Serdes.String();
                        -    private final Consumed stringConsumed = Consumed.with(stringSerde, stringSerde);
                        -    final private Serde intSerde = Serdes.Integer();
                        -    @Rule
                        -    public final KStreamTestDriver driver = new KStreamTestDriver();
                        -    private File stateDir = null;
                        -
                        -    @Before
                        -    public void setUp() {
                        -        stateDir = TestUtils.tempDirectory("kafka-test");
                        -    }
                        +    private final Consumed stringConsumed = Consumed.with(Serdes.String(), Serdes.String());
                        +    private final ConsumerRecordFactory recordFactory = new ConsumerRecordFactory<>(new StringSerializer(), new StringSerializer());
                        +    private final Properties props = StreamsTestUtils.topologyTestConfig(Serdes.String(), Serdes.String());
                         
                             @Test
                             public void testKTable() {
                        @@ -60,38 +56,38 @@ public void testKTable() {
                         
                                 final String topic1 = "topic1";
                         
                        -        final KTable table1 = builder.table(topic1, Consumed.with(stringSerde, intSerde));
                        +        final KTable table1 = builder.table(topic1, Consumed.with(Serdes.String(), Serdes.Integer()));
                         
                                 final MockProcessorSupplier supplier = new MockProcessorSupplier<>();
                                 table1.toStream().process(supplier);
                         
                        -        driver.setUp(builder, stateDir);
                        -        driver.process(topic1, "A", 1);
                        -        driver.process(topic1, "B", 2);
                        -        driver.process(topic1, "C", 3);
                        -        driver.process(topic1, "D", 4);
                        -        driver.flushState();
                        -        driver.process(topic1, "A", null);
                        -        driver.process(topic1, "B", null);
                        -        driver.flushState();
                        +        final ConsumerRecordFactory integerFactory = new ConsumerRecordFactory<>(new StringSerializer(), new IntegerSerializer());
                        +        try (final TopologyTestDriver driver = new TopologyTestDriver(builder.build(), props)) {
                        +            driver.pipeInput(integerFactory.create(topic1, "A", 1));
                        +            driver.pipeInput(integerFactory.create(topic1, "B", 2));
                        +            driver.pipeInput(integerFactory.create(topic1, "C", 3));
                        +            driver.pipeInput(integerFactory.create(topic1, "D", 4));
                        +            driver.pipeInput(integerFactory.create(topic1, "A", null));
                        +            driver.pipeInput(integerFactory.create(topic1, "B", null));
                        +        }
                         
                                 assertEquals(Utils.mkList("A:1", "B:2", "C:3", "D:4", "A:null", "B:null"), supplier.theCapturedProcessor().processed);
                             }
                         
                             @Test
                             public void kTableShouldLogAndMeterOnSkippedRecords() {
                        -        final StreamsBuilder streamsBuilder = new StreamsBuilder();
                        +        final StreamsBuilder builder = new StreamsBuilder();
                                 final String topic = "topic";
                        -        streamsBuilder.table(topic, Consumed.with(stringSerde, intSerde));
                        +        builder.table(topic, stringConsumed);
                         
                                 final LogCaptureAppender appender = LogCaptureAppender.createAndRegister();
                        -        driver.setUp(streamsBuilder, stateDir);
                        -        driver.process(topic, null, "value");
                        -        driver.flushState();
                        -        LogCaptureAppender.unregister(appender);
                        +        try (final TopologyTestDriver driver = new TopologyTestDriver(builder.build(), props)) {
                        +            driver.pipeInput(recordFactory.create(topic, null, "value"));
                        +            LogCaptureAppender.unregister(appender);
                         
                        -        assertEquals(1.0, getMetricByName(driver.context().metrics().metrics(), "skipped-records-total", "stream-metrics").metricValue());
                        -        assertThat(appender.getMessages(), hasItem("Skipping record due to null key. topic=[topic] partition=[-1] offset=[-1]"));
                        +            assertEquals(1.0, getMetricByName(driver.metrics(), "skipped-records-total", "stream-metrics").metricValue());
                        +            assertThat(appender.getMessages(), hasItem("Skipping record due to null key. topic=[topic] partition=[0] offset=[0]"));
                        +        }
                             }
                         
                             @Test
                        @@ -102,39 +98,45 @@ public void testValueGetter() {
                         
                                 final KTableImpl table1 = (KTableImpl) builder.table(topic1, stringConsumed);
                         
                        +        final Topology topology = builder.build();
                        +
                                 final KTableValueGetterSupplier getterSupplier1 = table1.valueGetterSupplier();
                         
                        -        driver.setUp(builder, stateDir);
                        -        final KTableValueGetter getter1 = getterSupplier1.get();
                        -        getter1.init(driver.context());
                        +        final InternalTopologyBuilder topologyBuilder = TopologyWrapper.getInternalTopologyBuilder(topology);
                        +        topologyBuilder.connectProcessorAndStateStores(table1.name, getterSupplier1.storeNames());
                        +
                        +        try (final TopologyTestDriverWrapper driver = new TopologyTestDriverWrapper(builder.build(), props)) {
                        +            final KTableValueGetter getter1 = getterSupplier1.get();
                        +            getter1.init(driver.setCurrentNodeForProcessorContext(table1.name));
                         
                        -        driver.process(topic1, "A", "01");
                        -        driver.process(topic1, "B", "01");
                        -        driver.process(topic1, "C", "01");
                        +            driver.pipeInput(recordFactory.create(topic1, "A", "01"));
                        +            driver.pipeInput(recordFactory.create(topic1, "B", "01"));
                        +            driver.pipeInput(recordFactory.create(topic1, "C", "01"));
                         
                        -        assertEquals("01", getter1.get("A"));
                        -        assertEquals("01", getter1.get("B"));
                        -        assertEquals("01", getter1.get("C"));
                        +            assertEquals("01", getter1.get("A"));
                        +            assertEquals("01", getter1.get("B"));
                        +            assertEquals("01", getter1.get("C"));
                         
                        -        driver.process(topic1, "A", "02");
                        -        driver.process(topic1, "B", "02");
                        +            driver.pipeInput(recordFactory.create(topic1, "A", "02"));
                        +            driver.pipeInput(recordFactory.create(topic1, "B", "02"));
                         
                        -        assertEquals("02", getter1.get("A"));
                        -        assertEquals("02", getter1.get("B"));
                        -        assertEquals("01", getter1.get("C"));
                        +            assertEquals("02", getter1.get("A"));
                        +            assertEquals("02", getter1.get("B"));
                        +            assertEquals("01", getter1.get("C"));
                         
                        -        driver.process(topic1, "A", "03");
                        +            driver.pipeInput(recordFactory.create(topic1, "A", "03"));
                         
                        -        assertEquals("03", getter1.get("A"));
                        -        assertEquals("02", getter1.get("B"));
                        -        assertEquals("01", getter1.get("C"));
                        +            assertEquals("03", getter1.get("A"));
                        +            assertEquals("02", getter1.get("B"));
                        +            assertEquals("01", getter1.get("C"));
                         
                        -        driver.process(topic1, "A", null);
                        -        driver.process(topic1, "B", null);
                        +            driver.pipeInput(recordFactory.create(topic1, "A", (String) null));
                        +            driver.pipeInput(recordFactory.create(topic1, "B", (String) null));
                         
                        -        assertNull(getter1.get("A"));
                        -        assertNull(getter1.get("B"));
                        -        assertEquals("01", getter1.get("C"));
                        +            assertNull(getter1.get("A"));
                        +            assertNull(getter1.get("B"));
                        +            assertEquals("01", getter1.get("C"));
                        +        }
                         
                             }
                         
                        @@ -148,35 +150,32 @@ public void testNotSendingOldValue() {
                         
                                 final MockProcessorSupplier supplier = new MockProcessorSupplier<>();
                         
                        -        builder.build().addProcessor("proc1", supplier, table1.name);
                        +        final Topology topology = builder.build().addProcessor("proc1", supplier, table1.name);
                         
                        -        driver.setUp(builder, stateDir);
                        +        try (final TopologyTestDriver driver = new TopologyTestDriver(topology, props)) {
                         
                        -        final MockProcessor proc1 = supplier.theCapturedProcessor();
                        +            final MockProcessor proc1 = supplier.theCapturedProcessor();
                         
                        -        driver.process(topic1, "A", "01");
                        -        driver.process(topic1, "B", "01");
                        -        driver.process(topic1, "C", "01");
                        -        driver.flushState();
                        +            driver.pipeInput(recordFactory.create(topic1, "A", "01"));
                        +            driver.pipeInput(recordFactory.create(topic1, "B", "01"));
                        +            driver.pipeInput(recordFactory.create(topic1, "C", "01"));
                         
                        -        proc1.checkAndClearProcessResult("A:(01<-null)", "B:(01<-null)", "C:(01<-null)");
                        +            proc1.checkAndClearProcessResult("A:(01<-null)", "B:(01<-null)", "C:(01<-null)");
                         
                        -        driver.process(topic1, "A", "02");
                        -        driver.process(topic1, "B", "02");
                        -        driver.flushState();
                        +            driver.pipeInput(recordFactory.create(topic1, "A", "02"));
                        +            driver.pipeInput(recordFactory.create(topic1, "B", "02"));
                         
                        -        proc1.checkAndClearProcessResult("A:(02<-null)", "B:(02<-null)");
                        +            proc1.checkAndClearProcessResult("A:(02<-null)", "B:(02<-null)");
                         
                        -        driver.process(topic1, "A", "03");
                        -        driver.flushState();
                        +            driver.pipeInput(recordFactory.create(topic1, "A", "03"));
                         
                        -        proc1.checkAndClearProcessResult("A:(03<-null)");
                        +            proc1.checkAndClearProcessResult("A:(03<-null)");
                         
                        -        driver.process(topic1, "A", null);
                        -        driver.process(topic1, "B", null);
                        -        driver.flushState();
                        +            driver.pipeInput(recordFactory.create(topic1, "A", (String) null));
                        +            driver.pipeInput(recordFactory.create(topic1, "B", (String) null));
                         
                        -        proc1.checkAndClearProcessResult("A:(null<-null)", "B:(null<-null)");
                        +            proc1.checkAndClearProcessResult("A:(null<-null)", "B:(null<-null)");
                        +        }
                             }
                         
                             @Test
                        @@ -193,34 +192,31 @@ public void testSendingOldValue() {
                         
                                 final MockProcessorSupplier supplier = new MockProcessorSupplier<>();
                         
                        -        builder.build().addProcessor("proc1", supplier, table1.name);
                        +        final Topology topology = builder.build().addProcessor("proc1", supplier, table1.name);
                         
                        -        driver.setUp(builder, stateDir);
                        +        try (final TopologyTestDriver driver = new TopologyTestDriver(topology, props)) {
                         
                        -        final MockProcessor proc1 = supplier.theCapturedProcessor();
                        +            final MockProcessor proc1 = supplier.theCapturedProcessor();
                         
                        -        driver.process(topic1, "A", "01");
                        -        driver.process(topic1, "B", "01");
                        -        driver.process(topic1, "C", "01");
                        -        driver.flushState();
                        +            driver.pipeInput(recordFactory.create(topic1, "A", "01"));
                        +            driver.pipeInput(recordFactory.create(topic1, "B", "01"));
                        +            driver.pipeInput(recordFactory.create(topic1, "C", "01"));
                         
                        -        proc1.checkAndClearProcessResult("A:(01<-null)", "B:(01<-null)", "C:(01<-null)");
                        +            proc1.checkAndClearProcessResult("A:(01<-null)", "B:(01<-null)", "C:(01<-null)");
                         
                        -        driver.process(topic1, "A", "02");
                        -        driver.process(topic1, "B", "02");
                        -        driver.flushState();
                        +            driver.pipeInput(recordFactory.create(topic1, "A", "02"));
                        +            driver.pipeInput(recordFactory.create(topic1, "B", "02"));
                         
                        -        proc1.checkAndClearProcessResult("A:(02<-01)", "B:(02<-01)");
                        +            proc1.checkAndClearProcessResult("A:(02<-01)", "B:(02<-01)");
                         
                        -        driver.process(topic1, "A", "03");
                        -        driver.flushState();
                        +            driver.pipeInput(recordFactory.create(topic1, "A", "03"));
                         
                        -        proc1.checkAndClearProcessResult("A:(03<-02)");
                        +            proc1.checkAndClearProcessResult("A:(03<-02)");
                         
                        -        driver.process(topic1, "A", null);
                        -        driver.process(topic1, "B", null);
                        -        driver.flushState();
                        +            driver.pipeInput(recordFactory.create(topic1, "A", (String) null));
                        +            driver.pipeInput(recordFactory.create(topic1, "B", (String) null));
                         
                        -        proc1.checkAndClearProcessResult("A:(null<-03)", "B:(null<-02)");
                        +            proc1.checkAndClearProcessResult("A:(null<-03)", "B:(null<-02)");
                        +        }
                             }
                         }
                        diff --git a/streams/src/test/java/org/apache/kafka/streams/perf/SimpleBenchmark.java b/streams/src/test/java/org/apache/kafka/streams/perf/SimpleBenchmark.java
                        index 8187467aaa61a..7179293200ed7 100644
                        --- a/streams/src/test/java/org/apache/kafka/streams/perf/SimpleBenchmark.java
                        +++ b/streams/src/test/java/org/apache/kafka/streams/perf/SimpleBenchmark.java
                        @@ -54,6 +54,7 @@
                         import org.apache.kafka.streams.state.WindowStore;
                         
                         import java.io.IOException;
                        +import java.time.Duration;
                         import java.util.ArrayList;
                         import java.util.List;
                         import java.util.Locale;
                        @@ -334,7 +335,7 @@ private void consumeAndProduce(final String topic) {
                                     consumer.seekToBeginning(partitions);
                         
                                     while (true) {
                        -                final ConsumerRecords records = consumer.poll(POLL_MS);
                        +                final ConsumerRecords records = consumer.poll(Duration.ofMillis(POLL_MS));
                                         if (records.isEmpty()) {
                                             if (processedRecords == numRecords) {
                                                 break;
                        @@ -372,7 +373,7 @@ private void consume(final String topic) {
                                     consumer.seekToBeginning(partitions);
                         
                                     while (true) {
                        -                final ConsumerRecords records = consumer.poll(POLL_MS);
                        +                final ConsumerRecords records = consumer.poll(Duration.ofMillis(POLL_MS));
                                         if (records.isEmpty()) {
                                             if (processedRecords == numRecords) {
                                                 break;
                        diff --git a/streams/src/test/java/org/apache/kafka/streams/processor/internals/AbstractTaskTest.java b/streams/src/test/java/org/apache/kafka/streams/processor/internals/AbstractTaskTest.java
                        index 347e9c4fd75c0..4ed44be47f2d0 100644
                        --- a/streams/src/test/java/org/apache/kafka/streams/processor/internals/AbstractTaskTest.java
                        +++ b/streams/src/test/java/org/apache/kafka/streams/processor/internals/AbstractTaskTest.java
                        @@ -42,6 +42,7 @@
                         
                         import java.io.File;
                         import java.io.IOException;
                        +import java.time.Duration;
                         import java.util.ArrayList;
                         import java.util.Collection;
                         import java.util.Collections;
                        @@ -233,7 +234,7 @@ private AbstractTask createTask(final Consumer consumer,
                                                         storeTopicPartitions,
                                                         ProcessorTopology.withLocalStores(new ArrayList<>(stateStoresToChangelogTopics.keySet()), storeNamesToChangelogTopics),
                                                         consumer,
                        -                                new StoreChangelogReader(consumer, new MockStateRestoreListener(), new LogContext("stream-task-test ")),
                        +                                new StoreChangelogReader(consumer, Duration.ZERO, new MockStateRestoreListener(), new LogContext("stream-task-test ")),
                                                         false,
                                                         stateDirectory,
                                                         config) {
                        diff --git a/streams/src/test/java/org/apache/kafka/streams/processor/internals/GlobalStateManagerImplTest.java b/streams/src/test/java/org/apache/kafka/streams/processor/internals/GlobalStateManagerImplTest.java
                        index 2ca9c211c1cda..8badf46fa665b 100644
                        --- a/streams/src/test/java/org/apache/kafka/streams/processor/internals/GlobalStateManagerImplTest.java
                        +++ b/streams/src/test/java/org/apache/kafka/streams/processor/internals/GlobalStateManagerImplTest.java
                        @@ -231,7 +231,7 @@ public void shouldRestoreRecordsUpToHighwatermark() {
                             @Test
                             public void shouldRecoverFromInvalidOffsetExceptionAndRestoreRecords() {
                                 initializeConsumer(2, 1, t1);
                        -        consumer.setException(new InvalidOffsetException("Try Again!") {
                        +        consumer.setPollException(new InvalidOffsetException("Try Again!") {
                                     public Set partitions() {
                                         return Collections.singleton(t1);
                                     }
                        @@ -488,6 +488,16 @@ public void shouldCheckpointRestoredOffsetsToFile() throws IOException {
                                 assertThat(readOffsetsCheckpoint(), equalTo(checkpointMap));
                             }
                         
                        +    @Test
                        +    public void shouldSkipGlobalInMemoryStoreOffsetsToFile() throws IOException {
                        +        stateManager.initialize();
                        +        initializeConsumer(10, 1, t3);
                        +        stateManager.register(store3, stateRestoreCallback);
                        +        stateManager.close(Collections.emptyMap());
                        +
                        +        assertThat(readOffsetsCheckpoint(), equalTo(Collections.emptyMap()));
                        +    }
                        +
                             private Map readOffsetsCheckpoint() throws IOException {
                                 final OffsetCheckpoint offsetCheckpoint = new OffsetCheckpoint(new File(stateManager.baseDir(),
                                                                                                         ProcessorStateManager.CHECKPOINT_FILE_NAME));
                        diff --git a/streams/src/test/java/org/apache/kafka/streams/processor/internals/GlobalStreamThreadTest.java b/streams/src/test/java/org/apache/kafka/streams/processor/internals/GlobalStreamThreadTest.java
                        index f3e9299cb509f..fded7c03ebc57 100644
                        --- a/streams/src/test/java/org/apache/kafka/streams/processor/internals/GlobalStreamThreadTest.java
                        +++ b/streams/src/test/java/org/apache/kafka/streams/processor/internals/GlobalStreamThreadTest.java
                        @@ -242,7 +242,7 @@ public boolean conditionMet() {
                                     }
                                 }, 10 * 1000, "Input record never consumed");
                         
                        -        mockConsumer.setException(new InvalidOffsetException("Try Again!") {
                        +        mockConsumer.setPollException(new InvalidOffsetException("Try Again!") {
                                     @Override
                                     public Set partitions() {
                                         return Collections.singleton(topicPartition);
                        diff --git a/streams/src/test/java/org/apache/kafka/streams/processor/internals/InternalTopologyBuilderTest.java b/streams/src/test/java/org/apache/kafka/streams/processor/internals/InternalTopologyBuilderTest.java
                        index 1da04255b1123..b0674ea338aef 100644
                        --- a/streams/src/test/java/org/apache/kafka/streams/processor/internals/InternalTopologyBuilderTest.java
                        +++ b/streams/src/test/java/org/apache/kafka/streams/processor/internals/InternalTopologyBuilderTest.java
                        @@ -159,6 +159,16 @@ public void testAddProcessorWithSelfParent() {
                                 builder.addProcessor("processor", new MockProcessorSupplier(), "processor");
                             }
                         
                        +    @Test(expected = TopologyException.class)
                        +    public void testAddProcessorWithEmptyParents() {
                        +        builder.addProcessor("processor", new MockProcessorSupplier());
                        +    }
                        +
                        +    @Test(expected = NullPointerException.class)
                        +    public void testAddProcessorWithNullParents() {
                        +        builder.addProcessor("processor", new MockProcessorSupplier(), null);
                        +    }
                        +
                             @Test
                             public void testAddSinkWithSameName() {
                                 builder.addSource(null, "source", null, null, null, "topic-1");
                        @@ -179,6 +189,17 @@ public void testAddSinkWithSelfParent() {
                                 builder.addSink("sink", "topic-2", null, null, null, "sink");
                             }
                         
                        +
                        +    @Test(expected = TopologyException.class)
                        +    public void testAddSinkWithEmptyParents() {
                        +        builder.addSink("sink", "topic", null, null, null);
                        +    }
                        +
                        +    @Test(expected = NullPointerException.class)
                        +    public void testAddSinkWithNullParents() {
                        +        builder.addSink("sink", "topic", null, null, null, null);
                        +    }
                        +
                             @Test
                             public void testAddSinkConnectedWithParent() {
                                 builder.addSource(null, "source", null, null, null, "source-topic");
                        @@ -275,7 +296,8 @@ public void testAddStateStoreWithSource() {
                         
                             @Test
                             public void testAddStateStoreWithSink() {
                        -        builder.addSink("sink-1", "topic-1", null, null, null);
                        +        builder.addSource(null, "source-1", null, null, null, "topic-1");
                        +        builder.addSink("sink-1", "topic-1", null, null, null, "source-1");
                                 try {
                                     builder.addStateStore(storeBuilder, "sink-1");
                                     fail("Should throw TopologyException with store cannot be added to sink");
                        diff --git a/streams/src/test/java/org/apache/kafka/streams/processor/internals/ProcessorNodeTest.java b/streams/src/test/java/org/apache/kafka/streams/processor/internals/ProcessorNodeTest.java
                        index 65dd0228b64c9..a64af9f8e7aed 100644
                        --- a/streams/src/test/java/org/apache/kafka/streams/processor/internals/ProcessorNodeTest.java
                        +++ b/streams/src/test/java/org/apache/kafka/streams/processor/internals/ProcessorNodeTest.java
                        @@ -92,7 +92,6 @@ public void testMetrics() {
                                 final InternalMockProcessorContext context = new InternalMockProcessorContext(
                                     anyStateSerde,
                                     new RecordCollectorImpl(
                        -                null,
                                         null,
                                         new LogContext("processnode-test "),
                                         new DefaultProductionExceptionHandler(),
                        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
                        index 6a20cd92b6b31..1b03cd4f2949a 100644
                        --- 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
                        @@ -123,7 +123,7 @@ public void shouldRestoreStoreWithBatchingRestoreSpecification() throws Exceptio
                                     assertThat(batchingRestoreCallback.getRestoredRecords().size(), is(1));
                                     assertTrue(batchingRestoreCallback.getRestoredRecords().contains(expectedKeyValue));
                                 } finally {
                        -            stateMgr.close(Collections.emptyMap());
                        +            stateMgr.close(Collections.emptyMap());
                                 }
                             }
                         
                        @@ -141,7 +141,7 @@ public void shouldRestoreStoreWithSinglePutRestoreSpecification() throws Excepti
                                     assertThat(persistentStore.keys.size(), is(1));
                                     assertTrue(persistentStore.keys.contains(intKey));
                                 } finally {
                        -            stateMgr.close(Collections.emptyMap());
                        +            stateMgr.close(Collections.emptyMap());
                                 }
                             }
                         
                        @@ -169,7 +169,7 @@ public void testRegisterPersistentStore() throws IOException {
                                     stateMgr.register(persistentStore, persistentStore.stateRestoreCallback);
                                     assertTrue(changelogReader.wasRegistered(new TopicPartition(persistentStoreTopicName, 2)));
                                 } finally {
                        -            stateMgr.close(Collections.emptyMap());
                        +            stateMgr.close(Collections.emptyMap());
                                 }
                             }
                         
                        @@ -196,7 +196,7 @@ public void testRegisterNonPersistentStore() throws IOException {
                                     stateMgr.register(nonPersistentStore, nonPersistentStore.stateRestoreCallback);
                                     assertTrue(changelogReader.wasRegistered(new TopicPartition(nonPersistentStoreTopicName, 2)));
                                 } finally {
                        -            stateMgr.close(Collections.emptyMap());
                        +            stateMgr.close(Collections.emptyMap());
                                 }
                             }
                         
                        @@ -257,7 +257,7 @@ public void testChangeLogOffsets() throws IOException {
                                     assertEquals(-1L, (long) changeLogOffsets.get(partition3));
                         
                                 } finally {
                        -            stateMgr.close(Collections.emptyMap());
                        +            stateMgr.close(Collections.emptyMap());
                                 }
                             }
                         
                        @@ -269,7 +269,7 @@ public void testGetStore() throws IOException {
                                     noPartitions,
                                     false,
                                     stateDirectory,
                        -            Collections.emptyMap(),
                        +            Collections.emptyMap(),
                                     changelogReader,
                                     false,
                                     logContext);
                        @@ -280,13 +280,13 @@ public void testGetStore() throws IOException {
                                     assertEquals(mockStateStore, stateMgr.getStore(nonPersistentStoreName));
                         
                                 } finally {
                        -            stateMgr.close(Collections.emptyMap());
                        +            stateMgr.close(Collections.emptyMap());
                                 }
                             }
                         
                             @Test
                             public void testFlushAndClose() throws IOException {
                        -        checkpoint.write(Collections.emptyMap());
                        +        checkpoint.write(Collections.emptyMap());
                         
                                 // set up ack'ed offsets
                                 final HashMap ackedOffsets = new HashMap<>();
                        @@ -339,7 +339,7 @@ public void shouldRegisterStoreWithoutLoggingEnabledAndNotBackedByATopic() throw
                                     noPartitions,
                                     false,
                                     stateDirectory,
                        -            Collections.emptyMap(),
                        +            Collections.emptyMap(),
                                     changelogReader,
                                     false,
                                     logContext);
                        @@ -358,7 +358,7 @@ public void shouldNotChangeOffsetsIfAckedOffsetsIsNull() throws IOException {
                                     noPartitions,
                                     false,
                                     stateDirectory,
                        -            Collections.emptyMap(),
                        +            Collections.emptyMap(),
                                     changelogReader,
                                     false,
                                     logContext);
                        @@ -408,7 +408,7 @@ public void shouldWriteCheckpointForStandbyReplica() throws IOException {
                                                                                           bytes,
                                                                                           bytes)));
                         
                        -        stateMgr.checkpoint(Collections.emptyMap());
                        +        stateMgr.checkpoint(Collections.emptyMap());
                         
                                 final Map read = checkpoint.read();
                                 assertThat(read, equalTo(Collections.singletonMap(persistentStorePartition, 889L)));
                        @@ -433,7 +433,7 @@ public void shouldNotWriteCheckpointForNonPersistent() throws IOException {
                                 stateMgr.checkpoint(Collections.singletonMap(topicPartition, 876L));
                         
                                 final Map read = checkpoint.read();
                        -        assertThat(read, equalTo(Collections.emptyMap()));
                        +        assertThat(read, equalTo(Collections.emptyMap()));
                             }
                         
                             @Test
                        @@ -443,7 +443,7 @@ public void shouldNotWriteCheckpointForStoresWithoutChangelogTopic() throws IOEx
                                     noPartitions,
                                     true, // standby
                                     stateDirectory,
                        -            Collections.emptyMap(),
                        +            Collections.emptyMap(),
                                     changelogReader,
                                     false,
                                     logContext);
                        @@ -453,10 +453,9 @@ public void shouldNotWriteCheckpointForStoresWithoutChangelogTopic() throws IOEx
                                 stateMgr.checkpoint(Collections.singletonMap(persistentStorePartition, 987L));
                         
                                 final Map read = checkpoint.read();
                        -        assertThat(read, equalTo(Collections.emptyMap()));
                        +        assertThat(read, equalTo(Collections.emptyMap()));
                             }
                         
                        -
                             @Test
                             public void shouldThrowIllegalArgumentExceptionIfStoreNameIsSameAsCheckpointFileName() throws IOException {
                                 final ProcessorStateManager stateManager = new ProcessorStateManager(
                        @@ -464,7 +463,7 @@ public void shouldThrowIllegalArgumentExceptionIfStoreNameIsSameAsCheckpointFile
                                     noPartitions,
                                     false,
                                     stateDirectory,
                        -            Collections.emptyMap(),
                        +            Collections.emptyMap(),
                                     changelogReader,
                                     false,
                                     logContext);
                        @@ -484,7 +483,7 @@ public void shouldThrowIllegalArgumentExceptionOnRegisterWhenStoreHasAlreadyBeen
                                     noPartitions,
                                     false,
                                     stateDirectory,
                        -            Collections.emptyMap(),
                        +            Collections.emptyMap(),
                                     changelogReader,
                                     false,
                                     logContext);
                        @@ -551,7 +550,7 @@ public void close() {
                                 stateManager.register(stateStore, stateStore.stateRestoreCallback);
                         
                                 try {
                        -            stateManager.close(Collections.emptyMap());
                        +            stateManager.close(Collections.emptyMap());
                                     fail("Should throw ProcessorStateException if store close throws exception");
                                 } catch (final ProcessorStateException e) {
                                     // pass
                        @@ -623,7 +622,7 @@ public void close() {
                                 stateManager.register(stateStore2, stateStore2.stateRestoreCallback);
                         
                                 try {
                        -            stateManager.close(Collections.emptyMap());
                        +            stateManager.close(Collections.emptyMap());
                                 } catch (final ProcessorStateException expected) { /* ignode */ }
                                 Assert.assertTrue(closedStore.get());
                             }
                        @@ -640,7 +639,7 @@ public void shouldDeleteCheckpointFileOnCreationIfEosEnabled() throws IOExceptio
                                         noPartitions,
                                         false,
                                         stateDirectory,
                        -                Collections.emptyMap(),
                        +                Collections.emptyMap(),
                                         changelogReader,
                                         true,
                                         logContext);
                        @@ -653,28 +652,36 @@ public void shouldDeleteCheckpointFileOnCreationIfEosEnabled() throws IOExceptio
                                 }
                             }
                         
                        -    @SuppressWarnings("unchecked")
                             @Test
                        -    public void shouldSuccessfullyReInitializeStateStores() throws IOException {
                        +    public void shouldSuccessfullyReInitializeStateStoresWithEosDisable() throws Exception {
                        +        shouldSuccessfullyReInitializeStateStores(false);
                        +    }
                        +
                        +    @Test
                        +    public void shouldSuccessfullyReInitializeStateStoresWithEosEnable() throws Exception {
                        +        shouldSuccessfullyReInitializeStateStores(true);
                        +    }
                        +
                        +    private void shouldSuccessfullyReInitializeStateStores(final boolean eosEnabled) throws Exception {
                                 final String store2Name = "store2";
                                 final String store2Changelog = "store2-changelog";
                                 final TopicPartition store2Partition = new TopicPartition(store2Changelog, 0);
                                 final List changelogPartitions = Arrays.asList(changelogTopicPartition, store2Partition);
                        -        Map storeToChangelog = new HashMap() {
                        +        final Map storeToChangelog = new HashMap() {
                                     {
                                         put(storeName, changelogTopic);
                                         put(store2Name, store2Changelog);
                                     }
                                 };
                                 final ProcessorStateManager stateManager = new ProcessorStateManager(
                        -                taskId,
                        -                changelogPartitions,
                        -                false,
                        -                stateDirectory,
                        -                storeToChangelog,
                        -                changelogReader,
                        -                false,
                        -                logContext);
                        +            taskId,
                        +            changelogPartitions,
                        +            false,
                        +            stateDirectory,
                        +            storeToChangelog,
                        +            changelogReader,
                        +            eosEnabled,
                        +            logContext);
                         
                                 final MockStateStore stateStore = new MockStateStore(storeName, true);
                                 final MockStateStore stateStore2 = new MockStateStore(store2Name, true);
                        @@ -696,7 +703,7 @@ public void register(final StateStore store, final StateRestoreCallback stateRes
                                 assertTrue(stateStore2.initialized);
                             }
                         
                        -    private ProcessorStateManager getStandByStateManager(TaskId taskId) throws IOException {
                        +    private ProcessorStateManager getStandByStateManager(final TaskId taskId) throws IOException {
                                 return new ProcessorStateManager(
                                     taskId,
                                     noPartitions,
                        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
                        index d88d3b5694afa..53a40c71b76a3 100644
                        --- 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
                        @@ -346,6 +346,32 @@ public void shouldConsiderModifiedTimeStamps() {
                                 assertNextOutputRecord(OUTPUT_TOPIC_1, "key3", "value3", partition, 40L);
                             }
                         
                        +    @Test
                        +    public void shouldConsiderModifiedTimeStampsForMultipleProcessors() {
                        +        final int partition = 10;
                        +        driver = new TopologyTestDriver(createMultiProcessorTimestampTopology(partition), props);
                        +
                        +        driver.pipeInput(recordFactory.create(INPUT_TOPIC_1, "key1", "value1", 10L));
                        +        assertNextOutputRecord(OUTPUT_TOPIC_1, "key1", "value1", partition, 10L);
                        +        assertNextOutputRecord(OUTPUT_TOPIC_2, "key1", "value1", partition, 20L);
                        +        assertNextOutputRecord(OUTPUT_TOPIC_1, "key1", "value1", partition, 15L);
                        +        assertNextOutputRecord(OUTPUT_TOPIC_2, "key1", "value1", partition, 20L);
                        +        assertNextOutputRecord(OUTPUT_TOPIC_1, "key1", "value1", partition, 12L);
                        +        assertNextOutputRecord(OUTPUT_TOPIC_2, "key1", "value1", partition, 22L);
                        +        assertNoOutputRecord(OUTPUT_TOPIC_1);
                        +        assertNoOutputRecord(OUTPUT_TOPIC_2);
                        +
                        +        driver.pipeInput(recordFactory.create(INPUT_TOPIC_1, "key2", "value2", 20L));
                        +        assertNextOutputRecord(OUTPUT_TOPIC_1, "key2", "value2", partition, 20L);
                        +        assertNextOutputRecord(OUTPUT_TOPIC_2, "key2", "value2", partition, 30L);
                        +        assertNextOutputRecord(OUTPUT_TOPIC_1, "key2", "value2", partition, 25L);
                        +        assertNextOutputRecord(OUTPUT_TOPIC_2, "key2", "value2", partition, 30L);
                        +        assertNextOutputRecord(OUTPUT_TOPIC_1, "key2", "value2", partition, 22L);
                        +        assertNextOutputRecord(OUTPUT_TOPIC_2, "key2", "value2", partition, 32L);
                        +        assertNoOutputRecord(OUTPUT_TOPIC_1);
                        +        assertNoOutputRecord(OUTPUT_TOPIC_2);
                        +    }
                        +
                             @Test
                             public void shouldConsiderHeaders() {
                                 final int partition = 10;
                        @@ -440,6 +466,16 @@ private Topology createTimestampTopology(final int partition) {
                                     .addSink("sink", OUTPUT_TOPIC_1, constantPartitioner(partition), "processor");
                             }
                         
                        +    private Topology createMultiProcessorTimestampTopology(final int partition) {
                        +        return topology
                        +            .addSource("source", STRING_DESERIALIZER, STRING_DESERIALIZER, INPUT_TOPIC_1)
                        +            .addProcessor("processor", define(new FanOutTimestampProcessor("child1", "child2")), "source")
                        +            .addProcessor("child1", define(new ForwardingProcessor()), "processor")
                        +            .addProcessor("child2", define(new TimestampProcessor()), "processor")
                        +            .addSink("sink1", OUTPUT_TOPIC_1, constantPartitioner(partition), "child1")
                        +            .addSink("sink2", OUTPUT_TOPIC_2, constantPartitioner(partition), "child2");
                        +    }
                        +
                             private Topology createMultiplexingTopology() {
                                 return topology
                                     .addSource("source", STRING_DESERIALIZER, STRING_DESERIALIZER, INPUT_TOPIC_1)
                        @@ -533,6 +569,25 @@ public void process(final String key, final String value) {
                                 }
                             }
                         
                        +    protected static class FanOutTimestampProcessor extends AbstractProcessor {
                        +        private final String firstChild;
                        +        private final String secondChild;
                        +
                        +        FanOutTimestampProcessor(final String firstChild,
                        +                                 final String secondChild) {
                        +            this.firstChild = firstChild;
                        +            this.secondChild = secondChild;
                        +        }
                        +
                        +        @Override
                        +        public void process(final String key, final String value) {
                        +            context().forward(key, value);
                        +            context().forward(key, value, To.child(firstChild).withTimestamp(context().timestamp() + 5));
                        +            context().forward(key, value, To.child(secondChild));
                        +            context().forward(key, value, To.all().withTimestamp(context().timestamp() + 2));
                        +        }
                        +    }
                        +
                             protected static class AddHeaderProcessor extends AbstractProcessor {
                                 @Override
                                 public void process(final String key, final String value) {
                        diff --git a/streams/src/test/java/org/apache/kafka/streams/processor/internals/RecordCollectorTest.java b/streams/src/test/java/org/apache/kafka/streams/processor/internals/RecordCollectorTest.java
                        index 6954eda529f74..92b94741379c5 100644
                        --- a/streams/src/test/java/org/apache/kafka/streams/processor/internals/RecordCollectorTest.java
                        +++ b/streams/src/test/java/org/apache/kafka/streams/processor/internals/RecordCollectorTest.java
                        @@ -50,6 +50,8 @@
                         import java.util.Map;
                         import java.util.concurrent.Future;
                         
                        +import static org.hamcrest.CoreMatchers.equalTo;
                        +import static org.hamcrest.MatcherAssert.assertThat;
                         import static org.junit.Assert.assertEquals;
                         import static org.junit.Assert.assertTrue;
                         import static org.junit.Assert.fail;
                        @@ -82,12 +84,12 @@ public Integer partition(final String topic, final String key, final Object valu
                             public void testSpecificPartition() {
                         
                                 final RecordCollectorImpl collector = new RecordCollectorImpl(
                        -            new MockProducer<>(cluster, true, new DefaultPartitioner(), byteArraySerializer, byteArraySerializer),
                                     "RecordCollectorTest-TestSpecificPartition",
                                     new LogContext("RecordCollectorTest-TestSpecificPartition "),
                                     new DefaultProductionExceptionHandler(),
                                     new Metrics().sensor("skipped-records")
                                 );
                        +        collector.init(new MockProducer<>(cluster, true, new DefaultPartitioner(), byteArraySerializer, byteArraySerializer));
                         
                                 final Headers headers = new RecordHeaders(new Header[]{new RecordHeader("key", "value".getBytes())});
                         
                        @@ -120,12 +122,12 @@ public void testSpecificPartition() {
                             public void testStreamPartitioner() {
                         
                                 final RecordCollectorImpl collector = new RecordCollectorImpl(
                        -            new MockProducer<>(cluster, true, new DefaultPartitioner(), byteArraySerializer, byteArraySerializer),
                                     "RecordCollectorTest-TestStreamPartitioner",
                                     new LogContext("RecordCollectorTest-TestStreamPartitioner "),
                                     new DefaultProductionExceptionHandler(),
                                     new Metrics().sensor("skipped-records")
                                 );
                        +        collector.init(new MockProducer<>(cluster, true, new DefaultPartitioner(), byteArraySerializer, byteArraySerializer));
                         
                                 final Headers headers = new RecordHeaders(new Header[]{new RecordHeader("key", "value".getBytes())});
                         
                        @@ -148,20 +150,48 @@ public void testStreamPartitioner() {
                                 assertEquals((Long) 0L, offsets.get(new TopicPartition("topic1", 2)));
                             }
                         
                        +    @Test
                        +    public void shouldNotAllowOffsetsToBeUpdatedExternally() {
                        +        final String topic = "topic1";
                        +        final TopicPartition topicPartition = new TopicPartition(topic, 0);
                        +
                        +        final RecordCollectorImpl collector = new RecordCollectorImpl(
                        +            "RecordCollectorTest-TestSpecificPartition",
                        +            new LogContext("RecordCollectorTest-TestSpecificPartition "),
                        +            new DefaultProductionExceptionHandler(),
                        +            new Metrics().sensor("skipped-records")
                        +        );
                        +        collector.init(new MockProducer<>(cluster, true, new DefaultPartitioner(), byteArraySerializer, byteArraySerializer));
                        +
                        +        collector.send(topic, "999", "0", null, 0, null, stringSerializer, stringSerializer);
                        +        collector.send(topic, "999", "0", null, 0, null, stringSerializer, stringSerializer);
                        +        collector.send(topic, "999", "0", null, 0, null, stringSerializer, stringSerializer);
                        +
                        +        final Map offsets = collector.offsets();
                        +
                        +        assertThat(offsets.get(topicPartition), equalTo(2L));
                        +        try {
                        +            offsets.put(new TopicPartition(topic, 0), 50L);
                        +            fail("Should have thrown UnsupportedOperationException");
                        +        } catch (final UnsupportedOperationException expected) { }
                        +
                        +        assertThat(collector.offsets().get(topicPartition), equalTo(2L));
                        +    }
                        +
                             @SuppressWarnings("unchecked")
                             @Test(expected = StreamsException.class)
                             public void shouldThrowStreamsExceptionOnAnyExceptionButProducerFencedException() {
                                 final RecordCollector collector = new RecordCollectorImpl(
                        -            new MockProducer(cluster, true, new DefaultPartitioner(), byteArraySerializer, byteArraySerializer) {
                        -                @Override
                        -                public synchronized Future send(final ProducerRecord record, final Callback callback) {
                        -                    throw new KafkaException();
                        -                }
                        -            },
                                     "test",
                                     logContext,
                                     new DefaultProductionExceptionHandler(),
                                     new Metrics().sensor("skipped-records"));
                        +        collector.init(new MockProducer(cluster, true, new DefaultPartitioner(), byteArraySerializer, byteArraySerializer) {
                        +            @Override
                        +            public synchronized Future send(final ProducerRecord record, final Callback callback) {
                        +                throw new KafkaException();
                        +            }
                        +        });
                         
                                 collector.send("topic1", "3", "0", null, null, stringSerializer, stringSerializer, streamPartitioner);
                             }
                        @@ -170,17 +200,17 @@ public synchronized Future send(final ProducerRecord record, fin
                             @Test
                             public void shouldThrowStreamsExceptionOnSubsequentCallIfASendFailsWithDefaultExceptionHandler() {
                                 final RecordCollector collector = new RecordCollectorImpl(
                        -            new MockProducer(cluster, true, new DefaultPartitioner(), byteArraySerializer, byteArraySerializer) {
                        -                @Override
                        -                public synchronized Future send(final ProducerRecord record, final Callback callback) {
                        -                    callback.onCompletion(null, new Exception());
                        -                    return null;
                        -                }
                        -            },
                                     "test",
                                     logContext,
                                     new DefaultProductionExceptionHandler(),
                                     new Metrics().sensor("skipped-records"));
                        +        collector.init(new MockProducer(cluster, true, new DefaultPartitioner(), byteArraySerializer, byteArraySerializer) {
                        +            @Override
                        +            public synchronized Future send(final ProducerRecord record, final Callback callback) {
                        +                callback.onCompletion(null, new Exception());
                        +                return null;
                        +            }
                        +        });
                         
                                 collector.send("topic1", "3", "0", null, null, stringSerializer, stringSerializer, streamPartitioner);
                         
                        @@ -194,17 +224,17 @@ public synchronized Future send(final ProducerRecord record, fin
                             @Test
                             public void shouldNotThrowStreamsExceptionOnSubsequentCallIfASendFailsWithContinueExceptionHandler() {
                                 final RecordCollector collector = new RecordCollectorImpl(
                        -            new MockProducer(cluster, true, new DefaultPartitioner(), byteArraySerializer, byteArraySerializer) {
                        -                @Override
                        -                public synchronized Future send(final ProducerRecord record, final Callback callback) {
                        -                    callback.onCompletion(null, new Exception());
                        -                    return null;
                        -                }
                        -            },
                                     "test",
                                     logContext,
                                     new AlwaysContinueProductionExceptionHandler(),
                                     new Metrics().sensor("skipped-records"));
                        +        collector.init(new MockProducer(cluster, true, new DefaultPartitioner(), byteArraySerializer, byteArraySerializer) {
                        +            @Override
                        +            public synchronized Future send(final ProducerRecord record, final Callback callback) {
                        +                callback.onCompletion(null, new Exception());
                        +                return null;
                        +            }
                        +        });
                         
                                 collector.send("topic1", "3", "0", null, null, stringSerializer, stringSerializer, streamPartitioner);
                         
                        @@ -220,17 +250,17 @@ public void shouldRecordSkippedMetricAndLogWarningIfSendFailsWithContinueExcepti
                                 final MetricName metricName = new MetricName("name", "group", "description", Collections.EMPTY_MAP);
                                 sensor.add(metricName, new Sum());
                                 final RecordCollector collector = new RecordCollectorImpl(
                        -            new MockProducer(cluster, true, new DefaultPartitioner(), byteArraySerializer, byteArraySerializer) {
                        -                @Override
                        -                public synchronized Future send(final ProducerRecord record, final Callback callback) {
                        -                    callback.onCompletion(null, new Exception());
                        -                    return null;
                        -                }
                        -            },
                                     "test",
                                     logContext,
                                     new AlwaysContinueProductionExceptionHandler(),
                                     sensor);
                        +        collector.init(new MockProducer(cluster, true, new DefaultPartitioner(), byteArraySerializer, byteArraySerializer) {
                        +            @Override
                        +            public synchronized Future send(final ProducerRecord record, final Callback callback) {
                        +                callback.onCompletion(null, new Exception());
                        +                return null;
                        +            }
                        +        });
                                 collector.send("topic1", "3", "0", null, null, stringSerializer, stringSerializer, streamPartitioner);
                                 assertEquals(1.0, metrics.metrics().get(metricName).metricValue());
                                 assertTrue(logCaptureAppender.getMessages().contains("test Error sending records (key=[3] value=[0] timestamp=[null]) to topic=[topic1] and partition=[0]; The exception handler chose to CONTINUE processing in spite of this error."));
                        @@ -241,17 +271,17 @@ public synchronized Future send(final ProducerRecord record, fin
                             @Test
                             public void shouldThrowStreamsExceptionOnFlushIfASendFailedWithDefaultExceptionHandler() {
                                 final RecordCollector collector = new RecordCollectorImpl(
                        -            new MockProducer(cluster, true, new DefaultPartitioner(), byteArraySerializer, byteArraySerializer) {
                        -                @Override
                        -                public synchronized Future send(final ProducerRecord record, final Callback callback) {
                        -                    callback.onCompletion(null, new Exception());
                        -                    return null;
                        -                }
                        -            },
                                     "test",
                                     logContext,
                                     new DefaultProductionExceptionHandler(),
                                     new Metrics().sensor("skipped-records"));
                        +        collector.init(new MockProducer(cluster, true, new DefaultPartitioner(), byteArraySerializer, byteArraySerializer) {
                        +            @Override
                        +            public synchronized Future send(final ProducerRecord record, final Callback callback) {
                        +                callback.onCompletion(null, new Exception());
                        +                return null;
                        +            }
                        +        });
                         
                                 collector.send("topic1", "3", "0", null, null, stringSerializer, stringSerializer, streamPartitioner);
                         
                        @@ -265,17 +295,17 @@ public synchronized Future send(final ProducerRecord record, fin
                             @Test
                             public void shouldNotThrowStreamsExceptionOnFlushIfASendFailedWithContinueExceptionHandler() {
                                 final RecordCollector collector = new RecordCollectorImpl(
                        -            new MockProducer(cluster, true, new DefaultPartitioner(), byteArraySerializer, byteArraySerializer) {
                        -                @Override
                        -                public synchronized Future send(final ProducerRecord record, final Callback callback) {
                        -                    callback.onCompletion(null, new Exception());
                        -                    return null;
                        -                }
                        -            },
                                     "test",
                                     logContext,
                                     new AlwaysContinueProductionExceptionHandler(),
                                     new Metrics().sensor("skipped-records"));
                        +        collector.init(new MockProducer(cluster, true, new DefaultPartitioner(), byteArraySerializer, byteArraySerializer) {
                        +            @Override
                        +            public synchronized Future send(final ProducerRecord record, final Callback callback) {
                        +                callback.onCompletion(null, new Exception());
                        +                return null;
                        +            }
                        +        });
                         
                                 collector.send("topic1", "3", "0", null, null, stringSerializer, stringSerializer, streamPartitioner);
                         
                        @@ -286,17 +316,17 @@ public synchronized Future send(final ProducerRecord record, fin
                             @Test
                             public void shouldThrowStreamsExceptionOnCloseIfASendFailedWithDefaultExceptionHandler() {
                                 final RecordCollector collector = new RecordCollectorImpl(
                        -            new MockProducer(cluster, true, new DefaultPartitioner(), byteArraySerializer, byteArraySerializer) {
                        -                @Override
                        -                public synchronized Future send(final ProducerRecord record, final Callback callback) {
                        -                    callback.onCompletion(null, new Exception());
                        -                    return null;
                        -                }
                        -            },
                                     "test",
                                     logContext,
                                     new DefaultProductionExceptionHandler(),
                                     new Metrics().sensor("skipped-records"));
                        +        collector.init(new MockProducer(cluster, true, new DefaultPartitioner(), byteArraySerializer, byteArraySerializer) {
                        +            @Override
                        +            public synchronized Future send(final ProducerRecord record, final Callback callback) {
                        +                callback.onCompletion(null, new Exception());
                        +                return null;
                        +            }
                        +        });
                         
                                 collector.send("topic1", "3", "0", null, null, stringSerializer, stringSerializer, streamPartitioner);
                         
                        @@ -310,17 +340,17 @@ public synchronized Future send(final ProducerRecord record, fin
                             @Test
                             public void shouldNotThrowStreamsExceptionOnCloseIfASendFailedWithContinueExceptionHandler() {
                                 final RecordCollector collector = new RecordCollectorImpl(
                        -            new MockProducer(cluster, true, new DefaultPartitioner(), byteArraySerializer, byteArraySerializer) {
                        -                @Override
                        -                public synchronized Future send(final ProducerRecord record, final Callback callback) {
                        -                    callback.onCompletion(null, new Exception());
                        -                    return null;
                        -                }
                        -            },
                                     "test",
                                     logContext,
                                     new AlwaysContinueProductionExceptionHandler(),
                                     new Metrics().sensor("skipped-records"));
                        +        collector.init(new MockProducer(cluster, true, new DefaultPartitioner(), byteArraySerializer, byteArraySerializer) {
                        +            @Override
                        +            public synchronized Future send(final ProducerRecord record, final Callback callback) {
                        +                callback.onCompletion(null, new Exception());
                        +                return null;
                        +            }
                        +        });
                         
                                 collector.send("topic1", "3", "0", null, null, stringSerializer, stringSerializer, streamPartitioner);
                         
                        @@ -331,17 +361,17 @@ public synchronized Future send(final ProducerRecord record, fin
                             @Test(expected = StreamsException.class)
                             public void shouldThrowIfTopicIsUnknownWithDefaultExceptionHandler() {
                                 final RecordCollector collector = new RecordCollectorImpl(
                        -            new MockProducer(cluster, true, new DefaultPartitioner(), byteArraySerializer, byteArraySerializer) {
                        -                @Override
                        -                public List partitionsFor(final String topic) {
                        -                    return Collections.EMPTY_LIST;
                        -                }
                        -
                        -            },
                                     "test",
                                     logContext,
                                     new DefaultProductionExceptionHandler(),
                                     new Metrics().sensor("skipped-records"));
                        +        collector.init(new MockProducer(cluster, true, new DefaultPartitioner(), byteArraySerializer, byteArraySerializer) {
                        +            @Override
                        +            public List partitionsFor(final String topic) {
                        +                return Collections.EMPTY_LIST;
                        +            }
                        +
                        +        });
                                 collector.send("topic1", "3", "0", null, null, stringSerializer, stringSerializer, streamPartitioner);
                             }
                         
                        @@ -349,17 +379,29 @@ public List partitionsFor(final String topic) {
                             @Test(expected = StreamsException.class)
                             public void shouldThrowIfTopicIsUnknownWithContinueExceptionHandler() {
                                 final RecordCollector collector = new RecordCollectorImpl(
                        -            new MockProducer(cluster, true, new DefaultPartitioner(), byteArraySerializer, byteArraySerializer) {
                        -                @Override
                        -                public List partitionsFor(final String topic) {
                        -                    return Collections.EMPTY_LIST;
                        -                }
                        -
                        -            },
                                     "test",
                                     logContext,
                                     new AlwaysContinueProductionExceptionHandler(),
                                     new Metrics().sensor("skipped-records"));
                        +        collector.init(new MockProducer(cluster, true, new DefaultPartitioner(), byteArraySerializer, byteArraySerializer) {
                        +            @Override
                        +            public List partitionsFor(final String topic) {
                        +                return Collections.EMPTY_LIST;
                        +            }
                        +
                        +        });
                                 collector.send("topic1", "3", "0", null, null, stringSerializer, stringSerializer, streamPartitioner);
                             }
                        +
                        +    @Test
                        +    public void testShouldNotThrowNPEOnCloseIfProducerIsNotInitialized() {
                        +        final RecordCollectorImpl collector = new RecordCollectorImpl(
                        +                "NoNPE",
                        +                logContext,
                        +                new DefaultProductionExceptionHandler(),
                        +                new Metrics().sensor("skipped-records")
                        +        );
                        +
                        +        collector.close();
                        +    }
                         }
                        diff --git a/streams/src/test/java/org/apache/kafka/streams/processor/internals/RecordDeserializerTest.java b/streams/src/test/java/org/apache/kafka/streams/processor/internals/RecordDeserializerTest.java
                        index 36988c0175b5d..2f2587cdedaf5 100644
                        --- a/streams/src/test/java/org/apache/kafka/streams/processor/internals/RecordDeserializerTest.java
                        +++ b/streams/src/test/java/org/apache/kafka/streams/processor/internals/RecordDeserializerTest.java
                        @@ -48,7 +48,7 @@ public class RecordDeserializerTest {
                         
                             @SuppressWarnings("deprecation")
                             @Test
                        -    public void shouldReturnNewConsumerRecordWithDeserializedValueWhenNoExceptions() {
                        +    public void shouldReturnConsumerRecordWithDeserializedValueWhenNoExceptions() {
                                 final RecordDeserializer recordDeserializer = new RecordDeserializer(
                                     new TheSourceNode(
                                         false,
                        @@ -105,4 +105,4 @@ public Object deserializeValue(final String topic, final Headers headers, final
                                 }
                             }
                         
                        -}
                        \ No newline at end of file
                        +}
                        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
                        index 3ed9e3b61a098..aab0dab333beb 100644
                        --- 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
                        @@ -61,7 +61,6 @@ public class RecordQueueTest {
                             final InternalMockProcessorContext context = new InternalMockProcessorContext(
                                 StateSerdes.withBuiltinTypes("anyName", Bytes.class, Bytes.class),
                                 new RecordCollectorImpl(
                        -            null,
                                     null,
                                     new LogContext("record-queue-test "),
                                     new DefaultProductionExceptionHandler(),
                        diff --git a/streams/src/test/java/org/apache/kafka/streams/processor/internals/SinkNodeTest.java b/streams/src/test/java/org/apache/kafka/streams/processor/internals/SinkNodeTest.java
                        index dacc17e86e7a6..269983f6380ff 100644
                        --- a/streams/src/test/java/org/apache/kafka/streams/processor/internals/SinkNodeTest.java
                        +++ b/streams/src/test/java/org/apache/kafka/streams/processor/internals/SinkNodeTest.java
                        @@ -37,20 +37,22 @@
                         public class SinkNodeTest {
                             private final Serializer anySerializer = Serdes.ByteArray().serializer();
                             private final StateSerdes anyStateSerde = StateSerdes.withBuiltinTypes("anyName", Bytes.class, Bytes.class);
                        +    private final RecordCollector recordCollector =  new RecordCollectorImpl(
                        +        null,
                        +        new LogContext("sinknode-test "),
                        +        new DefaultProductionExceptionHandler(),
                        +        new Metrics().sensor("skipped-records")
                        +    );
                        +
                             private final InternalMockProcessorContext context = new InternalMockProcessorContext(
                                 anyStateSerde,
                        -        new RecordCollectorImpl(
                        -            new MockProducer<>(true, anySerializer, anySerializer),
                        -            null,
                        -            new LogContext("sinknode-test "),
                        -            new DefaultProductionExceptionHandler(),
                        -            new Metrics().sensor("skipped-records")
                        -        )
                        +        recordCollector
                             );
                             private final SinkNode sink = new SinkNode<>("anyNodeName", new StaticTopicNameExtractor("any-output-topic"), anySerializer, anySerializer, null);
                         
                             @Before
                             public void before() {
                        +        recordCollector.init(new MockProducer<>(true, anySerializer, anySerializer));
                                 sink.init(context);
                             }
                         
                        diff --git a/streams/src/test/java/org/apache/kafka/streams/processor/internals/StandbyTaskTest.java b/streams/src/test/java/org/apache/kafka/streams/processor/internals/StandbyTaskTest.java
                        index 93d6a0d931bb0..05d0e3d04eede 100644
                        --- a/streams/src/test/java/org/apache/kafka/streams/processor/internals/StandbyTaskTest.java
                        +++ b/streams/src/test/java/org/apache/kafka/streams/processor/internals/StandbyTaskTest.java
                        @@ -50,6 +50,7 @@
                         
                         import java.io.File;
                         import java.io.IOException;
                        +import java.time.Duration;
                         import java.util.ArrayList;
                         import java.util.Arrays;
                         import java.util.Collections;
                        @@ -122,7 +123,12 @@ private StreamsConfig createConfig(final File baseDir) throws IOException {
                         
                             private final MockConsumer consumer = new MockConsumer<>(OffsetResetStrategy.EARLIEST);
                             private final MockRestoreConsumer restoreStateConsumer = new MockRestoreConsumer();
                        -    private final StoreChangelogReader changelogReader = new StoreChangelogReader(restoreStateConsumer, stateRestoreListener, new LogContext("standby-task-test "));
                        +    private final StoreChangelogReader changelogReader = new StoreChangelogReader(
                        +        restoreStateConsumer,
                        +        Duration.ZERO,
                        +        stateRestoreListener,
                        +        new LogContext("standby-task-test ")
                        +    );
                         
                             private final byte[] recordValue = intSerializer.serialize(null, 10);
                             private final byte[] recordKey = intSerializer.serialize(null, 1);
                        @@ -188,7 +194,7 @@ public void testUpdate() throws IOException {
                                 }
                         
                                 restoreStateConsumer.seekToBeginning(partition);
                        -        task.update(partition2, restoreStateConsumer.poll(100).records(partition2));
                        +        task.update(partition2, restoreStateConsumer.poll(Duration.ofMillis(100)).records(partition2));
                         
                                 StandbyContextImpl context = (StandbyContextImpl) task.context();
                                 MockStateStore store1 = (MockStateStore) context.getStateMgr().getStore(storeName1);
                        @@ -245,7 +251,7 @@ public void testUpdateKTable() throws IOException {
                                 }
                         
                                 // The commit offset is at 0L. Records should not be processed
                        -        List> remaining = task.update(globalTopicPartition, restoreStateConsumer.poll(100).records(globalTopicPartition));
                        +        List> remaining = task.update(globalTopicPartition, restoreStateConsumer.poll(Duration.ofMillis(100)).records(globalTopicPartition));
                                 assertEquals(5, remaining.size());
                         
                                 committedOffsets.put(new TopicPartition(globalTopicPartition.topic(), globalTopicPartition.partition()), new OffsetAndMetadata(10L));
                        diff --git a/streams/src/test/java/org/apache/kafka/streams/processor/internals/StateConsumerTest.java b/streams/src/test/java/org/apache/kafka/streams/processor/internals/StateConsumerTest.java
                        index 725211dd268dc..140f705619956 100644
                        --- a/streams/src/test/java/org/apache/kafka/streams/processor/internals/StateConsumerTest.java
                        +++ b/streams/src/test/java/org/apache/kafka/streams/processor/internals/StateConsumerTest.java
                        @@ -27,6 +27,7 @@
                         import org.junit.Test;
                         
                         import java.io.IOException;
                        +import java.time.Duration;
                         import java.util.HashMap;
                         import java.util.Map;
                         
                        @@ -52,7 +53,7 @@ public void setUp() {
                                 partitionOffsets.put(topicOne, 20L);
                                 partitionOffsets.put(topicTwo, 30L);
                                 stateMaintainer = new StateMaintainerStub(partitionOffsets);
                        -        stateConsumer = new GlobalStreamThread.StateConsumer(logContext, consumer, stateMaintainer, time, 10L, FLUSH_INTERVAL);
                        +        stateConsumer = new GlobalStreamThread.StateConsumer(logContext, consumer, stateMaintainer, time, Duration.ofMillis(10L), FLUSH_INTERVAL);
                             }
                         
                             @Test
                        @@ -109,7 +110,7 @@ public void shouldNotFlushOffsetsWhenFlushIntervalHasNotLapsed() {
                         
                             @Test
                             public void shouldNotFlushWhenFlushIntervalIsZero() {
                        -        stateConsumer = new GlobalStreamThread.StateConsumer(logContext, consumer, stateMaintainer, time, 10L, -1);
                        +        stateConsumer = new GlobalStreamThread.StateConsumer(logContext, consumer, stateMaintainer, time, Duration.ofMillis(10L), -1);
                                 stateConsumer.initialize();
                                 time.sleep(100);
                                 stateConsumer.pollAndUpdate();
                        diff --git a/streams/src/test/java/org/apache/kafka/streams/processor/internals/StoreChangelogReaderTest.java b/streams/src/test/java/org/apache/kafka/streams/processor/internals/StoreChangelogReaderTest.java
                        index aabe7ff631362..729efc2d94c2e 100644
                        --- a/streams/src/test/java/org/apache/kafka/streams/processor/internals/StoreChangelogReaderTest.java
                        +++ b/streams/src/test/java/org/apache/kafka/streams/processor/internals/StoreChangelogReaderTest.java
                        @@ -39,6 +39,7 @@
                         import org.junit.Test;
                         import org.junit.runner.RunWith;
                         
                        +import java.time.Duration;
                         import java.util.Collection;
                         import java.util.Collections;
                         import java.util.List;
                        @@ -71,7 +72,7 @@ public class StoreChangelogReaderTest {
                             private final StateRestoreListener stateRestoreListener = new MockStateRestoreListener();
                             private final TopicPartition topicPartition = new TopicPartition("topic", 0);
                             private final LogContext logContext = new LogContext("test-reader ");
                        -    private final StoreChangelogReader changelogReader = new StoreChangelogReader(consumer, stateRestoreListener, logContext);
                        +    private final StoreChangelogReader changelogReader = new StoreChangelogReader(consumer, Duration.ZERO, stateRestoreListener, logContext);
                         
                             @Before
                             public void setUp() {
                        @@ -89,7 +90,7 @@ public Map> listTopics() {
                                     }
                                 };
                         
                        -        final StoreChangelogReader changelogReader = new StoreChangelogReader(consumer, stateRestoreListener, logContext);
                        +        final StoreChangelogReader changelogReader = new StoreChangelogReader(consumer, Duration.ZERO, stateRestoreListener, logContext);
                                 changelogReader.register(new StateRestorer(topicPartition, restoreListener, null, Long.MAX_VALUE, true, "storeName"));
                                 changelogReader.restore(active);
                                 assertTrue(functionCalled.get());
                        @@ -99,7 +100,11 @@ public Map> listTopics() {
                             public void shouldThrowExceptionIfConsumerHasCurrentSubscription() {
                                 final StateRestorer mockRestorer = EasyMock.mock(StateRestorer.class);
                                 mockRestorer.setUserRestoreListener(stateRestoreListener);
                        -        expect(mockRestorer.partition()).andReturn(new TopicPartition("sometopic", 0)).andReturn(new TopicPartition("sometopic", 0));
                        +        expect(mockRestorer.partition())
                        +                .andReturn(new TopicPartition("sometopic", 0))
                        +                .andReturn(new TopicPartition("sometopic", 0))
                        +                .andReturn(new TopicPartition("sometopic", 0))
                        +                .andReturn(new TopicPartition("sometopic", 0));
                                 EasyMock.replay(mockRestorer);
                                 changelogReader.register(mockRestorer);
                         
                        @@ -118,7 +123,10 @@ public void shouldRestoreAllMessagesFromBeginningWhenCheckpointNull() {
                                 final int messages = 10;
                                 setupConsumer(messages, topicPartition);
                                 changelogReader.register(new StateRestorer(topicPartition, restoreListener, null, Long.MAX_VALUE, true, "storeName"));
                        +        expect(active.restoringTaskFor(topicPartition)).andStubReturn(task);
                        +        replay(active, task);
                                 changelogReader.restore(active);
                        +
                                 assertThat(callback.restored.size(), equalTo(messages));
                             }
                         
                        @@ -126,7 +134,7 @@ public void shouldRestoreAllMessagesFromBeginningWhenCheckpointNull() {
                             public void shouldRecoverFromInvalidOffsetExceptionAndFinishRestore() {
                                 final int messages = 10;
                                 setupConsumer(messages, topicPartition);
                        -        consumer.setException(new InvalidOffsetException("Try Again!") {
                        +        consumer.setPollException(new InvalidOffsetException("Try Again!") {
                                     @Override
                                     public Set partitions() {
                                         return Collections.singleton(topicPartition);
                        @@ -135,16 +143,55 @@ public Set partitions() {
                                 changelogReader.register(new StateRestorer(topicPartition, restoreListener, null, Long.MAX_VALUE, true,
                                                                            "storeName"));
                         
                        -        EasyMock.expect(active.restoringTaskFor(topicPartition)).andReturn(task);
                        -        EasyMock.replay(active);
                        +        EasyMock.expect(active.restoringTaskFor(topicPartition)).andStubReturn(task);
                        +        EasyMock.replay(active, task);
                         
                                 // first restore call "fails" but we should not die with an exception
                                 assertEquals(0, changelogReader.restore(active).size());
                        +
                        +        changelogReader.register(new StateRestorer(topicPartition, restoreListener, null, Long.MAX_VALUE, true,
                        +                "storeName"));
                                 // retry restore should succeed
                                 assertEquals(1, changelogReader.restore(active).size());
                                 assertThat(callback.restored.size(), equalTo(messages));
                             }
                         
                        +    @Test
                        +    public void shouldRecoverFromOffsetOutOfRangeExceptionAndRestoreFromStart() {
                        +        final int messages = 10;
                        +        final int startOffset = 5;
                        +        final long expiredCheckpoint = 1L;
                        +        assignPartition(messages, topicPartition);
                        +        consumer.updateBeginningOffsets(Collections.singletonMap(topicPartition, (long) startOffset));
                        +        consumer.updateEndOffsets(Collections.singletonMap(topicPartition, (long) (messages + startOffset)));
                        +
                        +        addRecords(messages, topicPartition, startOffset);
                        +        consumer.assign(Collections.emptyList());
                        +
                        +        final StateRestorer stateRestorer = new StateRestorer(
                        +                topicPartition,
                        +                restoreListener,
                        +                expiredCheckpoint,
                        +                Long.MAX_VALUE,
                        +                true,
                        +                "storeName");
                        +        changelogReader.register(stateRestorer);
                        +
                        +        EasyMock.expect(active.restoringTaskFor(topicPartition)).andStubReturn(task);
                        +        EasyMock.replay(active, task);
                        +
                        +        // first restore call "fails" since OffsetOutOfRangeException but we should not die with an exception
                        +        assertEquals(0, changelogReader.restore(active).size());
                        +        //the starting offset for stateRestorer is set to NO_CHECKPOINT
                        +        assertThat(stateRestorer.checkpoint(), equalTo(-1L));
                        +
                        +        //restore the active task again
                        +        changelogReader.register(stateRestorer);
                        +        //the restored task should return completed partition without Exception.
                        +        assertEquals(1, changelogReader.restore(active).size());
                        +        //the restored size should be equal to message length.
                        +        assertThat(callback.restored.size(), equalTo(messages));
                        +    }
                         
                             @Test
                             public void shouldRestoreMessagesFromCheckpoint() {
                        @@ -163,7 +210,8 @@ public void shouldClearAssignmentAtEndOfRestore() {
                                 setupConsumer(messages, topicPartition);
                                 changelogReader.register(new StateRestorer(topicPartition, restoreListener, null, Long.MAX_VALUE, true,
                                                                            "storeName"));
                        -
                        +        expect(active.restoringTaskFor(topicPartition)).andStubReturn(task);
                        +        replay(active, task);
                                 changelogReader.restore(active);
                                 assertThat(consumer.assignment(), equalTo(Collections.emptySet()));
                             }
                        @@ -174,6 +222,8 @@ public void shouldRestoreToLimitWhenSupplied() {
                                 final StateRestorer restorer = new StateRestorer(topicPartition, restoreListener, null, 3, true,
                                                                                  "storeName");
                                 changelogReader.register(restorer);
                        +        expect(active.restoringTaskFor(topicPartition)).andStubReturn(task);
                        +        replay(active, task);
                                 changelogReader.restore(active);
                                 assertThat(callback.restored.size(), equalTo(3));
                                 assertThat(restorer.restoredOffset(), equalTo(3L));
                        @@ -191,14 +241,14 @@ public void shouldRestoreMultipleStores() {
                                 setupConsumer(5, one);
                                 setupConsumer(3, two);
                         
                        -        changelogReader
                        -            .register(new StateRestorer(topicPartition, restoreListener, null, Long.MAX_VALUE, true, "storeName1"));
                        +        changelogReader.register(new StateRestorer(topicPartition, restoreListener, null, Long.MAX_VALUE, true, "storeName1"));
                                 changelogReader.register(new StateRestorer(one, restoreListener1, null, Long.MAX_VALUE, true, "storeName2"));
                                 changelogReader.register(new StateRestorer(two, restoreListener2, null, Long.MAX_VALUE, true, "storeName3"));
                         
                        -        expect(active.restoringTaskFor(one)).andReturn(null);
                        -        expect(active.restoringTaskFor(two)).andReturn(null);
                        -        replay(active);
                        +        expect(active.restoringTaskFor(one)).andStubReturn(task);
                        +        expect(active.restoringTaskFor(two)).andStubReturn(task);
                        +        expect(active.restoringTaskFor(topicPartition)).andStubReturn(task);
                        +        replay(active, task);
                                 changelogReader.restore(active);
                         
                                 assertThat(callback.restored.size(), equalTo(10));
                        @@ -219,13 +269,17 @@ public void shouldRestoreAndNotifyMultipleStores() throws Exception {
                                 setupConsumer(3, two);
                         
                                 changelogReader
                        -            .register(new StateRestorer(topicPartition, restoreListener, null, Long.MAX_VALUE, true, "storeName1"));
                        -        changelogReader.register(new StateRestorer(one, restoreListener1, null, Long.MAX_VALUE, true, "storeName2"));
                        -        changelogReader.register(new StateRestorer(two, restoreListener2, null, Long.MAX_VALUE, true, "storeName3"));
                        +            .register(new StateRestorer(topicPartition, restoreListener, 0L, Long.MAX_VALUE, true, "storeName1"));
                        +        changelogReader.register(new StateRestorer(one, restoreListener1, 0L, Long.MAX_VALUE, true, "storeName2"));
                        +        changelogReader.register(new StateRestorer(two, restoreListener2, 0L, Long.MAX_VALUE, true, "storeName3"));
                        +
                        +        expect(active.restoringTaskFor(one)).andReturn(task);
                        +        expect(active.restoringTaskFor(two)).andReturn(task);
                        +        expect(active.restoringTaskFor(topicPartition)).andReturn(task);
                        +        expect(active.restoringTaskFor(topicPartition)).andStubReturn(task);
                        +        replay(active, task);
                        +        changelogReader.restore(active);
                         
                        -        expect(active.restoringTaskFor(one)).andReturn(null);
                        -        expect(active.restoringTaskFor(two)).andReturn(null);
                        -        replay(active);
                                 changelogReader.restore(active);
                         
                                 assertThat(callback.restored.size(), equalTo(10));
                        @@ -246,7 +300,9 @@ public void shouldRestoreAndNotifyMultipleStores() throws Exception {
                             public void shouldOnlyReportTheLastRestoredOffset() {
                                 setupConsumer(10, topicPartition);
                                 changelogReader
                        -            .register(new StateRestorer(topicPartition, restoreListener, null, 5, true, "storeName1"));
                        +            .register(new StateRestorer(topicPartition, restoreListener, 0L, 5, true, "storeName1"));
                        +        expect(active.restoringTaskFor(topicPartition)).andStubReturn(task);
                        +        replay(active, task);
                                 changelogReader.restore(active);
                         
                                 assertThat(callback.restored.size(), equalTo(5));
                        @@ -305,7 +361,10 @@ public void shouldNotRestoreAnythingWhenCheckpointAtEndOffset() {
                             public void shouldReturnRestoredOffsetsForPersistentStores() {
                                 setupConsumer(10, topicPartition);
                                 changelogReader.register(new StateRestorer(topicPartition, restoreListener, null, Long.MAX_VALUE, true, "storeName"));
                        +        expect(active.restoringTaskFor(topicPartition)).andStubReturn(task);
                        +        replay(active, task);
                                 changelogReader.restore(active);
                        +
                                 final Map restoredOffsets = changelogReader.restoredOffsets();
                                 assertThat(restoredOffsets, equalTo(Collections.singletonMap(topicPartition, 10L)));
                             }
                        @@ -314,6 +373,8 @@ public void shouldReturnRestoredOffsetsForPersistentStores() {
                             public void shouldNotReturnRestoredOffsetsForNonPersistentStore() {
                                 setupConsumer(10, topicPartition);
                                 changelogReader.register(new StateRestorer(topicPartition, restoreListener, null, Long.MAX_VALUE, false, "storeName"));
                        +        expect(active.restoringTaskFor(topicPartition)).andStubReturn(task);
                        +        replay(active, task);
                                 changelogReader.restore(active);
                                 final Map restoredOffsets = changelogReader.restoredOffsets();
                                 assertThat(restoredOffsets, equalTo(Collections.emptyMap()));
                        @@ -329,6 +390,8 @@ public void shouldIgnoreNullKeysWhenRestoring() {
                                 consumer.assign(Collections.singletonList(topicPartition));
                                 changelogReader.register(new StateRestorer(topicPartition, restoreListener, null, Long.MAX_VALUE, false,
                                                                            "storeName"));
                        +        expect(active.restoringTaskFor(topicPartition)).andStubReturn(task);
                        +        replay(active, task);
                                 changelogReader.restore(active);
                         
                                 assertThat(callback.restored, CoreMatchers.equalTo(Utils.mkList(KeyValue.pair(bytes, bytes), KeyValue.pair(bytes, bytes))));
                        @@ -339,6 +402,9 @@ public void shouldCompleteImmediatelyWhenEndOffsetIs0() {
                                 final Collection expected = Collections.singleton(topicPartition);
                                 setupConsumer(0, topicPartition);
                                 changelogReader.register(new StateRestorer(topicPartition, restoreListener, null, Long.MAX_VALUE, true, "store"));
                        +        expect(active.restoringTaskFor(topicPartition)).andStubReturn(task);
                        +        replay(active, task);
                        +
                                 final Collection restored = changelogReader.restore(active);
                                 assertThat(restored, equalTo(expected));
                             }
                        @@ -353,10 +419,9 @@ public void shouldRestorePartitionsRegisteredPostInitialization() {
                                 changelogReader.register(new StateRestorer(topicPartition, restoreListener, null, Long.MAX_VALUE, false, "storeName"));
                         
                                 final TopicPartition postInitialization = new TopicPartition("other", 0);
                        -        expect(active.restoringTaskFor(topicPartition)).andReturn(null);
                        -        expect(active.restoringTaskFor(topicPartition)).andReturn(null);
                        -        expect(active.restoringTaskFor(postInitialization)).andReturn(null);
                        -        replay(active);
                        +        expect(active.restoringTaskFor(topicPartition)).andStubReturn(task);
                        +        expect(active.restoringTaskFor(postInitialization)).andStubReturn(task);
                        +        replay(active, task);
                         
                                 assertTrue(changelogReader.restore(active).isEmpty());
                         
                        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
                        index 3a0fc4eb1cdd9..41d36109bb3da 100644
                        --- 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
                        @@ -22,6 +22,8 @@
                         import org.apache.kafka.clients.producer.MockProducer;
                         import org.apache.kafka.common.KafkaException;
                         import org.apache.kafka.common.TopicPartition;
                        +import org.apache.kafka.common.errors.ProducerFencedException;
                        +import org.apache.kafka.common.errors.TimeoutException;
                         import org.apache.kafka.common.metrics.KafkaMetric;
                         import org.apache.kafka.common.metrics.Metrics;
                         import org.apache.kafka.common.metrics.Sensor;
                        @@ -35,13 +37,16 @@
                         import org.apache.kafka.common.utils.MockTime;
                         import org.apache.kafka.common.utils.Utils;
                         import org.apache.kafka.streams.StreamsConfig;
                        +import org.apache.kafka.streams.errors.DefaultProductionExceptionHandler;
                         import org.apache.kafka.streams.errors.StreamsException;
                        +import org.apache.kafka.streams.errors.TaskMigratedException;
                         import org.apache.kafka.streams.processor.PunctuationType;
                         import org.apache.kafka.streams.processor.Punctuator;
                         import org.apache.kafka.streams.processor.StateRestoreListener;
                         import org.apache.kafka.streams.processor.StateStore;
                         import org.apache.kafka.streams.processor.TaskId;
                         import org.apache.kafka.streams.processor.internals.metrics.StreamsMetricsImpl;
                        +import org.apache.kafka.streams.processor.internals.testutil.LogCaptureAppender;
                         import org.apache.kafka.streams.state.internals.OffsetCheckpoint;
                         import org.apache.kafka.test.MockProcessorNode;
                         import org.apache.kafka.test.MockSourceNode;
                        @@ -56,17 +61,21 @@
                         
                         import java.io.File;
                         import java.io.IOException;
                        -import java.util.Arrays;
                        +import java.time.Duration;
                         import java.util.Collections;
                        +import java.util.List;
                         import java.util.Map;
                         import java.util.Set;
                         import java.util.concurrent.atomic.AtomicBoolean;
                        +import java.util.stream.Collectors;
                         
                        +import static java.util.Arrays.asList;
                         import static java.util.Collections.singletonList;
                         import static org.apache.kafka.common.utils.Utils.mkEntry;
                         import static org.apache.kafka.common.utils.Utils.mkMap;
                         import static org.apache.kafka.common.utils.Utils.mkProperties;
                         import static org.hamcrest.CoreMatchers.equalTo;
                        +import static org.hamcrest.CoreMatchers.is;
                         import static org.hamcrest.CoreMatchers.nullValue;
                         import static org.hamcrest.MatcherAssert.assertThat;
                         import static org.junit.Assert.assertEquals;
                        @@ -113,10 +122,10 @@ public void close() {
                             );
                         
                             private final MockConsumer consumer = new MockConsumer<>(OffsetResetStrategy.EARLIEST);
                        -    private final MockProducer producer = new MockProducer<>(false, bytesSerializer, bytesSerializer);
                        +    private MockProducer producer;
                             private final MockConsumer restoreStateConsumer = new MockConsumer<>(OffsetResetStrategy.EARLIEST);
                             private final StateRestoreListener stateRestoreListener = new MockStateRestoreListener();
                        -    private final StoreChangelogReader changelogReader = new StoreChangelogReader(restoreStateConsumer, stateRestoreListener, new LogContext("stream-task-test ")) {
                        +    private final StoreChangelogReader changelogReader = new StoreChangelogReader(restoreStateConsumer, Duration.ZERO, stateRestoreListener, new LogContext("stream-task-test ")) {
                                 @Override
                                 public Map restoredOffsets() {
                                     return Collections.singletonMap(changelogPartition, offset);
                        @@ -160,7 +169,7 @@ private StreamsConfig createConfig(final boolean enableEoS) {
                         
                             @Before
                             public void setup() {
                        -        consumer.assign(Arrays.asList(partition1, partition2));
                        +        consumer.assign(asList(partition1, partition2));
                                 stateDirectory = new StateDirectory(createConfig(false), new MockTime());
                             }
                         
                        @@ -179,18 +188,142 @@ public void cleanup() throws IOException {
                                 }
                             }
                         
                        +    @Test
                        +    public void shouldHandleInitTransactionsTimeoutExceptionOnCreation() {
                        +        final LogCaptureAppender appender = LogCaptureAppender.createAndRegister();
                        +
                        +        final ProcessorTopology topology = ProcessorTopology.withSources(
                        +            asList(source1, source2, processorStreamTime, processorSystemTime),
                        +            mkMap(mkEntry(topic1, (SourceNode) source1), mkEntry(topic2, (SourceNode) source2))
                        +        );
                        +
                        +        source1.addChild(processorStreamTime);
                        +        source2.addChild(processorStreamTime);
                        +        source1.addChild(processorSystemTime);
                        +        source2.addChild(processorSystemTime);
                        +
                        +        try {
                        +            new StreamTask(
                        +                taskId00,
                        +                partitions,
                        +                topology,
                        +                consumer,
                        +                changelogReader,
                        +                createConfig(true),
                        +                streamsMetrics,
                        +                stateDirectory,
                        +                null,
                        +                time,
                        +                () -> producer = new MockProducer(false, bytesSerializer, bytesSerializer) {
                        +                    @Override
                        +                    public void initTransactions() {
                        +                        throw new TimeoutException("test");
                        +                    }
                        +                }
                        +            );
                        +            fail("Expected an exception");
                        +        } catch (final StreamsException expected) {
                        +            // make sure we log the explanation as an ERROR
                        +            assertTimeoutErrorLog(appender);
                        +
                        +            // make sure we report the correct message
                        +            assertThat(expected.getMessage(), is("task [0_0] Failed to initialize task 0_0 due to timeout."));
                        +
                        +            // make sure we preserve the cause
                        +            assertEquals(expected.getCause().getClass(), TimeoutException.class);
                        +            assertThat(expected.getCause().getMessage(), is("test"));
                        +        }
                        +        LogCaptureAppender.unregister(appender);
                        +    }
                        +
                        +    @Test
                        +    public void shouldHandleInitTransactionsTimeoutExceptionOnResume() {
                        +        final LogCaptureAppender appender = LogCaptureAppender.createAndRegister();
                        +
                        +        final ProcessorTopology topology = ProcessorTopology.withSources(
                        +            asList(source1, source2, processorStreamTime, processorSystemTime),
                        +            mkMap(mkEntry(topic1, (SourceNode) source1), mkEntry(topic2, (SourceNode) source2))
                        +        );
                        +
                        +        source1.addChild(processorStreamTime);
                        +        source2.addChild(processorStreamTime);
                        +        source1.addChild(processorSystemTime);
                        +        source2.addChild(processorSystemTime);
                        +
                        +        final AtomicBoolean timeOut = new AtomicBoolean(false);
                        +
                        +        final StreamTask testTask = new StreamTask(
                        +            taskId00,
                        +            partitions,
                        +            topology,
                        +            consumer,
                        +            changelogReader,
                        +            createConfig(true),
                        +            streamsMetrics,
                        +            stateDirectory,
                        +            null,
                        +            time,
                        +            () -> producer = new MockProducer(false, bytesSerializer, bytesSerializer) {
                        +                @Override
                        +                public void initTransactions() {
                        +                    if (timeOut.get()) {
                        +                        throw new TimeoutException("test");
                        +                    } else {
                        +                        super.initTransactions();
                        +                    }
                        +                }
                        +            }
                        +        );
                        +        testTask.initializeTopology();
                        +        testTask.suspend();
                        +        timeOut.set(true);
                        +        try {
                        +            testTask.resume();
                        +            fail("Expected an exception");
                        +        } catch (final StreamsException expected) {
                        +            // make sure we log the explanation as an ERROR
                        +            assertTimeoutErrorLog(appender);
                        +
                        +            // make sure we report the correct message
                        +            assertThat(expected.getMessage(), is("task [0_0] Failed to initialize task 0_0 due to timeout."));
                        +
                        +            // make sure we preserve the cause
                        +            assertEquals(expected.getCause().getClass(), TimeoutException.class);
                        +            assertThat(expected.getCause().getMessage(), is("test"));
                        +        }
                        +        LogCaptureAppender.unregister(appender);
                        +    }
                        +
                        +    private void assertTimeoutErrorLog(final LogCaptureAppender appender) {
                        +
                        +        final String expectedErrorLogMessage =
                        +            "task [0_0] Timeout exception caught when initializing transactions for task 0_0. " +
                        +                "This might happen if the broker is slow to respond, if the network " +
                        +                "connection to the broker was interrupted, or if similar circumstances arise. " +
                        +                "You can increase producer parameter `max.block.ms` to increase this timeout.";
                        +
                        +        final List expectedError =
                        +            appender
                        +                .getEvents()
                        +                .stream()
                        +                .filter(event -> event.getMessage().equals(expectedErrorLogMessage))
                        +                .map(LogCaptureAppender.Event::getLevel)
                        +                .collect(Collectors.toList());
                        +        assertThat(expectedError, is(singletonList("ERROR")));
                        +    }
                        +
                             @SuppressWarnings("unchecked")
                             @Test
                             public void testProcessOrder() {
                                 task = createStatelessTask(createConfig(false));
                         
                        -        task.addRecords(partition1, Arrays.asList(
                        +        task.addRecords(partition1, asList(
                                     getConsumerRecord(partition1, 10),
                                     getConsumerRecord(partition1, 20),
                                     getConsumerRecord(partition1, 30)
                                 ));
                         
                        -        task.addRecords(partition2, Arrays.asList(
                        +        task.addRecords(partition2, asList(
                                     getConsumerRecord(partition2, 25),
                                     getConsumerRecord(partition2, 35),
                                     getConsumerRecord(partition2, 45)
                        @@ -255,12 +388,12 @@ private KafkaMetric getMetric(final String nameFormat, final String descriptionF
                             public void testPauseResume() {
                                 task = createStatelessTask(createConfig(false));
                         
                        -        task.addRecords(partition1, Arrays.asList(
                        +        task.addRecords(partition1, asList(
                                     getConsumerRecord(partition1, 10),
                                     getConsumerRecord(partition1, 20)
                                 ));
                         
                        -        task.addRecords(partition2, Arrays.asList(
                        +        task.addRecords(partition2, asList(
                                     getConsumerRecord(partition2, 35),
                                     getConsumerRecord(partition2, 45),
                                     getConsumerRecord(partition2, 55),
                        @@ -274,7 +407,7 @@ public void testPauseResume() {
                                 assertEquals(1, consumer.paused().size());
                                 assertTrue(consumer.paused().contains(partition2));
                         
                        -        task.addRecords(partition1, Arrays.asList(
                        +        task.addRecords(partition1, asList(
                                     getConsumerRecord(partition1, 30),
                                     getConsumerRecord(partition1, 40),
                                     getConsumerRecord(partition1, 50)
                        @@ -312,7 +445,7 @@ public void testMaybePunctuateStreamTime() {
                                 task.initializeStateStores();
                                 task.initializeTopology();
                         
                        -        task.addRecords(partition1, Arrays.asList(
                        +        task.addRecords(partition1, asList(
                                     getConsumerRecord(partition1, 0),
                                     getConsumerRecord(partition1, 20),
                                     getConsumerRecord(partition1, 32),
                        @@ -320,7 +453,7 @@ public void testMaybePunctuateStreamTime() {
                                     getConsumerRecord(partition1, 60)
                                 ));
                         
                        -        task.addRecords(partition2, Arrays.asList(
                        +        task.addRecords(partition2, asList(
                                     getConsumerRecord(partition2, 25),
                                     getConsumerRecord(partition2, 35),
                                     getConsumerRecord(partition2, 45),
                        @@ -403,14 +536,14 @@ public void shouldPunctuateOnceStreamTimeAfterGap() {
                                 task.initializeStateStores();
                                 task.initializeTopology();
                         
                        -        task.addRecords(partition1, Arrays.asList(
                        +        task.addRecords(partition1, asList(
                                     getConsumerRecord(partition1, 20),
                                     getConsumerRecord(partition1, 142),
                                     getConsumerRecord(partition1, 155),
                                     getConsumerRecord(partition1, 160)
                                 ));
                         
                        -        task.addRecords(partition2, Arrays.asList(
                        +        task.addRecords(partition2, asList(
                                     getConsumerRecord(partition2, 25),
                                     getConsumerRecord(partition2, 145),
                                     getConsumerRecord(partition2, 159),
                        @@ -489,13 +622,13 @@ public void testCancelPunctuateStreamTime() {
                                 task.initializeStateStores();
                                 task.initializeTopology();
                         
                        -        task.addRecords(partition1, Arrays.asList(
                        +        task.addRecords(partition1, asList(
                                     getConsumerRecord(partition1, 20),
                                     getConsumerRecord(partition1, 30),
                                     getConsumerRecord(partition1, 40)
                                 ));
                         
                        -        task.addRecords(partition2, Arrays.asList(
                        +        task.addRecords(partition2, asList(
                                     getConsumerRecord(partition2, 25),
                                     getConsumerRecord(partition2, 35),
                                     getConsumerRecord(partition2, 45)
                        @@ -589,7 +722,7 @@ public void testCancelPunctuateSystemTime() {
                         
                             @Test
                             public void shouldWrapKafkaExceptionsWithStreamsExceptionAndAddContext() {
                        -        task = createTaskThatThrowsException();
                        +        task = createTaskThatThrowsException(false);
                                 task.initializeStateStores();
                                 task.initializeTopology();
                                 task.addRecords(partition2, singletonList(getConsumerRecord(partition2, 0)));
                        @@ -659,7 +792,7 @@ public void shouldFlushRecordCollectorOnFlushState() {
                                     stateDirectory,
                                     null,
                                     time,
                        -            producer,
                        +            () -> producer = new MockProducer<>(false, bytesSerializer, bytesSerializer),
                                     new NoOpRecordCollector() {
                                         @Override
                                         public void flush() {
                        @@ -755,15 +888,191 @@ public void punctuate(final long timestamp) {
                                 });
                             }
                         
                        +    @Test
                        +    public void shouldNotCloseProducerOnCleanCloseWithEosDisabled() {
                        +        task = createStatelessTask(createConfig(false));
                        +        task.close(true, false);
                        +        task = null;
                        +
                        +        assertFalse(producer.closed());
                        +    }
                        +
                        +    @Test
                        +    public void shouldNotCloseProducerOnUncleanCloseWithEosDisabled() {
                        +        task = createStatelessTask(createConfig(false));
                        +        task.close(false, false);
                        +        task = null;
                        +
                        +        assertFalse(producer.closed());
                        +    }
                        +
                        +    @Test
                        +    public void shouldNotCloseProducerOnErrorDuringCleanCloseWithEosDisabled() {
                        +        task = createTaskThatThrowsException(false);
                        +
                        +        try {
                        +            task.close(true, false);
                        +            fail("should have thrown runtime exception");
                        +        } catch (final RuntimeException expected) {
                        +            task = null;
                        +        }
                        +
                        +        assertFalse(producer.closed());
                        +    }
                        +
                        +    @Test
                        +    public void shouldNotCloseProducerOnErrorDuringUncleanCloseWithEosDisabled() {
                        +        task = createTaskThatThrowsException(false);
                        +
                        +        task.close(false, false);
                        +        task = null;
                        +
                        +        assertFalse(producer.closed());
                        +    }
                        +
                        +    @Test
                        +    public void shouldCommitTransactionAndCloseProducerOnCleanCloseWithEosEnabled() {
                        +        task = createStatelessTask(createConfig(true));
                        +        task.initializeTopology();
                        +
                        +        task.close(true, false);
                        +        task = null;
                        +
                        +        assertTrue(producer.transactionCommitted());
                        +        assertFalse(producer.transactionInFlight());
                        +        assertTrue(producer.closed());
                        +    }
                        +
                        +    @Test
                        +    public void shouldNotAbortTransactionAndNotCloseProducerOnErrorDuringCleanCloseWithEosEnabled() {
                        +        task = createTaskThatThrowsException(true);
                        +        task.initializeTopology();
                        +
                        +        try {
                        +            task.close(true, false);
                        +            fail("should have thrown runtime exception");
                        +        } catch (final RuntimeException expected) {
                        +            task = null;
                        +        }
                        +
                        +        assertTrue(producer.transactionInFlight());
                        +        assertFalse(producer.closed());
                        +    }
                        +
                        +    @Test
                        +    public void shouldOnlyCloseProducerIfFencedOnCommitDuringCleanCloseWithEosEnabled() {
                        +        task = createStatelessTask(createConfig(true));
                        +        task.initializeTopology();
                        +        producer.fenceProducer();
                        +
                        +        try {
                        +            task.close(true, false);
                        +            fail("should have thrown TaskMigratedException");
                        +        } catch (final TaskMigratedException expected) {
                        +            task = null;
                        +            assertTrue(expected.getCause() instanceof ProducerFencedException);
                        +        }
                        +
                        +        assertFalse(producer.transactionCommitted());
                        +        assertTrue(producer.transactionInFlight());
                        +        assertFalse(producer.transactionAborted());
                        +        assertFalse(producer.transactionCommitted());
                        +        assertTrue(producer.closed());
                        +    }
                        +
                        +    @Test
                        +    public void shouldNotCloseProducerIfFencedOnCloseDuringCleanCloseWithEosEnabled() {
                        +        task = createStatelessTask(createConfig(true));
                        +        task.initializeTopology();
                        +        producer.fenceProducerOnClose();
                        +
                        +        try {
                        +            task.close(true, false);
                        +            fail("should have thrown TaskMigratedException");
                        +        } catch (final TaskMigratedException expected) {
                        +            task = null;
                        +            assertTrue(expected.getCause() instanceof ProducerFencedException);
                        +        }
                        +
                        +        assertTrue(producer.transactionCommitted());
                        +        assertFalse(producer.transactionInFlight());
                        +        assertFalse(producer.closed());
                        +    }
                        +
                        +    @Test
                        +    public void shouldAbortTransactionAndCloseProducerOnUncleanCloseWithEosEnabled() {
                        +        task = createStatelessTask(createConfig(true));
                        +        task.initializeTopology();
                        +
                        +        task.close(false, false);
                        +        task = null;
                        +
                        +        assertTrue(producer.transactionAborted());
                        +        assertFalse(producer.transactionInFlight());
                        +        assertTrue(producer.closed());
                        +    }
                        +
                        +    @Test
                        +    public void shouldAbortTransactionAndCloseProducerOnErrorDuringUncleanCloseWithEosEnabled() {
                        +        task = createTaskThatThrowsException(true);
                        +        task.initializeTopology();
                        +
                        +        task.close(false, false);
                        +
                        +        assertTrue(producer.transactionAborted());
                        +        assertTrue(producer.closed());
                        +    }
                        +
                        +    @Test
                        +    public void shouldOnlyCloseProducerIfFencedOnAbortDuringUncleanCloseWithEosEnabled() {
                        +        task = createStatelessTask(createConfig(true));
                        +        task.initializeTopology();
                        +        producer.fenceProducer();
                        +
                        +        task.close(false, false);
                        +        task = null;
                        +
                        +        assertTrue(producer.transactionInFlight());
                        +        assertFalse(producer.transactionAborted());
                        +        assertFalse(producer.transactionCommitted());
                        +        assertTrue(producer.closed());
                        +    }
                        +
                        +    @Test
                        +    public void shouldOnlyCloseFencedProducerOnUncleanClosedWithEosEnabled() {
                        +        task = createStatelessTask(createConfig(true));
                        +        task.initializeTopology();
                        +        producer.fenceProducer();
                        +
                        +        task.close(false, true);
                        +        task = null;
                        +
                        +        assertFalse(producer.transactionAborted());
                        +        assertTrue(producer.closed());
                        +    }
                        +
                        +    @Test
                        +    public void shouldAbortTransactionButNotCloseProducerIfFencedOnCloseDuringUncleanCloseWithEosEnabled() {
                        +        task = createStatelessTask(createConfig(true));
                        +        task.initializeTopology();
                        +        producer.fenceProducerOnClose();
                        +
                        +        task.close(false, false);
                        +        task = null;
                        +
                        +        assertTrue(producer.transactionAborted());
                        +        assertFalse(producer.closed());
                        +    }
                        +
                             @Test
                             public void shouldThrowExceptionIfAnyExceptionsRaisedDuringCloseButStillCloseAllProcessorNodesTopology() {
                        -        task = createTaskThatThrowsException();
                        +        task = createTaskThatThrowsException(false);
                                 task.initializeStateStores();
                                 task.initializeTopology();
                                 try {
                                     task.close(true, false);
                                     fail("should have thrown runtime exception");
                        -        } catch (final RuntimeException e) {
                        +        } catch (final RuntimeException expected) {
                                     task = null;
                                 }
                                 assertTrue(processorSystemTime.closed);
                        @@ -780,11 +1089,24 @@ public void shouldInitAndBeginTransactionOnCreateIfEosEnabled() {
                                 assertTrue(producer.transactionInFlight());
                             }
                         
                        +    @Test
                        +    public void shouldWrapProducerFencedExceptionWithTaskMigragedExceptionForBeginTransaction() {
                        +        task = createStatelessTask(createConfig(true));
                        +        producer.fenceProducer();
                        +
                        +        try {
                        +            task.initializeTopology();
                        +            fail("Should have throws TaskMigratedException");
                        +        } catch (final TaskMigratedException expected) {
                        +            assertTrue(expected.getCause() instanceof ProducerFencedException);
                        +        }
                        +    }
                        +
                             @Test
                             public void shouldNotThrowOnCloseIfTaskWasNotInitializedWithEosEnabled() {
                                 task = createStatelessTask(createConfig(true));
                         
                        -        assertTrue(!producer.transactionInFlight());
                        +        assertFalse(producer.transactionInFlight());
                                 task.close(false, false);
                             }
                         
                        @@ -832,6 +1154,37 @@ public void shouldNotSendOffsetsAndCommitTransactionNorStartNewTransactionOnSusp
                                 assertFalse(producer.transactionInFlight());
                             }
                         
                        +    @Test
                        +    public void shouldWrapProducerFencedExceptionWithTaskMigragedExceptionInSuspendWhenCommitting() {
                        +        task = createStatelessTask(createConfig(true));
                        +        producer.fenceProducer();
                        +
                        +        try {
                        +            task.suspend();
                        +            fail("Should have throws TaskMigratedException");
                        +        } catch (final TaskMigratedException expected) {
                        +            assertTrue(expected.getCause() instanceof ProducerFencedException);
                        +        }
                        +
                        +        assertFalse(producer.transactionCommitted());
                        +    }
                        +
                        +    @Test
                        +    public void shouldWrapProducerFencedExceptionWithTaskMigragedExceptionInSuspendWhenClosingProducer() {
                        +        task = createStatelessTask(createConfig(true));
                        +        task.initializeTopology();
                        +
                        +        producer.fenceProducerOnClose();
                        +        try {
                        +            task.suspend();
                        +            fail("Should have throws TaskMigratedException");
                        +        } catch (final TaskMigratedException expected) {
                        +            assertTrue(expected.getCause() instanceof ProducerFencedException);
                        +        }
                        +
                        +        assertTrue(producer.transactionCommitted());
                        +    }
                        +
                             @Test
                             public void shouldStartNewTransactionOnResumeIfEosEnabled() {
                                 task = createStatelessTask(createConfig(true));
                        @@ -881,16 +1234,6 @@ public void shouldNotStartNewTransactionOnCommitIfEosDisabled() {
                                 assertFalse(producer.transactionInFlight());
                             }
                         
                        -    @Test
                        -    public void shouldAbortTransactionOnDirtyClosedIfEosEnabled() {
                        -        task = createStatelessTask(createConfig(true));
                        -        task.initializeTopology();
                        -        task.close(false, false);
                        -        task = null;
                        -
                        -        assertTrue(producer.transactionAborted());
                        -    }
                        -
                             @Test
                             public void shouldNotAbortTransactionOnZombieClosedIfEosEnabled() {
                                 task = createStatelessTask(createConfig(true));
                        @@ -912,15 +1255,36 @@ public void shouldNotAbortTransactionOnDirtyClosedIfEosDisabled() {
                             @Test
                             public void shouldCloseProducerOnCloseWhenEosEnabled() {
                                 task = createStatelessTask(createConfig(true));
                        +        task.initializeTopology();
                                 task.close(true, false);
                                 task = null;
                         
                                 assertTrue(producer.closed());
                             }
                         
                        +    @Test
                        +    public void shouldCloseProducerOnUncleanCloseNotZombieWhenEosEnabled() {
                        +        task = createStatelessTask(createConfig(true));
                        +        task.initializeTopology();
                        +        task.close(false, false);
                        +        task = null;
                        +
                        +        assertTrue(producer.closed());
                        +    }
                        +
                        +    @Test
                        +    public void shouldCloseProducerOnUncleanCloseIsZombieWhenEosEnabled() {
                        +        task = createStatelessTask(createConfig(true));
                        +        task.initializeTopology();
                        +        task.close(false, true);
                        +        task = null;
                        +
                        +        assertTrue(producer.closed());
                        +    }
                        +
                             @Test
                             public void shouldNotViolateAtLeastOnceWhenExceptionOccursDuringFlushing() {
                        -        task = createTaskThatThrowsException();
                        +        task = createTaskThatThrowsException(false);
                                 task.initializeStateStores();
                                 task.initializeTopology();
                         
                        @@ -934,7 +1298,7 @@ public void shouldNotViolateAtLeastOnceWhenExceptionOccursDuringFlushing() {
                         
                             @Test
                             public void shouldNotViolateAtLeastOnceWhenExceptionOccursDuringTaskSuspension() {
                        -        final StreamTask task = createTaskThatThrowsException();
                        +        final StreamTask task = createTaskThatThrowsException(false);
                         
                                 task.initializeStateStores();
                                 task.initializeTopology();
                        @@ -965,7 +1329,7 @@ public void shouldCloseStateManagerIfFailureOnTaskClose() {
                         
                             @Test
                             public void shouldNotCloseTopologyProcessorNodesIfNotInitialized() {
                        -        final StreamTask task = createTaskThatThrowsException();
                        +        final StreamTask task = createTaskThatThrowsException(false);
                                 try {
                                     task.close(false, false);
                                 } catch (final Exception e) {
                        @@ -996,7 +1360,7 @@ public void shouldReturnOffsetsForRepartitionTopicsForPurging() {
                                     mkMap(mkEntry(topic1, (SourceNode) source1), mkEntry(repartition.topic(), (SourceNode) source2)),
                                     Collections.singleton(repartition.topic())
                                 );
                        -        consumer.assign(Arrays.asList(partition1, repartition));
                        +        consumer.assign(asList(partition1, repartition));
                         
                                 task = new StreamTask(
                                     taskId00,
                        @@ -1009,7 +1373,7 @@ public void shouldReturnOffsetsForRepartitionTopicsForPurging() {
                                     stateDirectory,
                                     null,
                                     time,
                        -            producer
                        +            () -> producer = new MockProducer<>(false, bytesSerializer, bytesSerializer)
                                 );
                                 task.initializeStateStores();
                                 task.initializeTopology();
                        @@ -1027,6 +1391,41 @@ public void shouldReturnOffsetsForRepartitionTopicsForPurging() {
                                 assertThat(map, equalTo(Collections.singletonMap(repartition, 11L)));
                             }
                         
                        +    @Test
                        +    public void shouldThrowOnCleanCloseTaskWhenEosEnabledIfTransactionInFlight() {
                        +        task = createStatelessTask(createConfig(true));
                        +        try {
                        +            task.close(true, false);
                        +            fail("should have throw IllegalStateException");
                        +        } catch (final IllegalStateException expected) {
                        +            // pass
                        +        }
                        +        task = null;
                        +
                        +        assertTrue(producer.closed());
                        +    }
                        +
                        +    @Test
                        +    public void shouldAlwaysCommitIfEosEnabled() {
                        +        task = createStatelessTask(createConfig(true));
                        +
                        +        final RecordCollectorImpl recordCollector =  new RecordCollectorImpl("StreamTask",
                        +                new LogContext("StreamTaskTest "), new DefaultProductionExceptionHandler(), new Metrics().sensor("skipped-records"));
                        +        recordCollector.init(producer);
                        +
                        +        task.initializeStateStores();
                        +        task.initializeTopology();
                        +        task.punctuate(processorSystemTime, 5, PunctuationType.WALL_CLOCK_TIME, new Punctuator() {
                        +            @Override
                        +            public void punctuate(final long timestamp) {
                        +                recordCollector.send("result-topic1", 3, 5, null, 0, time.milliseconds(),
                        +                        new IntegerSerializer(),  new IntegerSerializer());
                        +            }
                        +        });
                        +        task.commit();
                        +        assertEquals(1, producer.history().size());
                        +    }
                        +
                             private StreamTask createStatefulTask(final StreamsConfig config, final boolean logged) {
                                 final ProcessorTopology topology = ProcessorTopology.with(
                                     Utils.mkList(source1, source2),
                        @@ -1045,7 +1444,7 @@ private StreamTask createStatefulTask(final StreamsConfig config, final boolean
                                     stateDirectory,
                                     null,
                                     time,
                        -            producer
                        +            () -> producer = new MockProducer<>(false, bytesSerializer, bytesSerializer)
                                 );
                             }
                         
                        @@ -1067,7 +1466,7 @@ private StreamTask createStatefulTaskThatThrowsExceptionOnClose() {
                                     stateDirectory,
                                     null,
                                     time,
                        -            producer
                        +            () -> producer = new MockProducer<>(false, bytesSerializer, bytesSerializer)
                                 );
                             }
                         
                        @@ -1093,12 +1492,12 @@ private StreamTask createStatelessTask(final StreamsConfig streamsConfig) {
                                     stateDirectory,
                                     null,
                                     time,
                        -            producer
                        +            () -> producer = new MockProducer<>(false, bytesSerializer, bytesSerializer)
                                 );
                             }
                         
                             // this task will throw exception when processing (on partition2), flushing, suspending and closing
                        -    private StreamTask createTaskThatThrowsException() {
                        +    private StreamTask createTaskThatThrowsException(final boolean enableEos) {
                                 final ProcessorTopology topology = ProcessorTopology.withSources(
                                     Utils.mkList(source1, source3, processorStreamTime, processorSystemTime),
                                     mkMap(mkEntry(topic1, (SourceNode) source1), mkEntry(topic2, (SourceNode) source3))
                        @@ -1115,12 +1514,12 @@ private StreamTask createTaskThatThrowsException() {
                                     topology,
                                     consumer,
                                     changelogReader,
                        -            createConfig(false),
                        +            createConfig(enableEos),
                                     streamsMetrics,
                                     stateDirectory,
                                     null,
                                     time,
                        -            producer
                        +            () -> producer = new MockProducer<>(false, bytesSerializer, bytesSerializer)
                                 ) {
                                     @Override
                                     protected void flushState() {
                        @@ -1143,5 +1542,4 @@ private ConsumerRecord getConsumerRecord(final TopicPartition to
                                     recordValue
                                 );
                             }
                        -
                         }
                        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
                        index 936c67b8ef838..daef49aad33ae 100644
                        --- 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
                        @@ -21,7 +21,7 @@
                         import org.apache.kafka.clients.consumer.ConsumerRecord;
                         import org.apache.kafka.clients.consumer.InvalidOffsetException;
                         import org.apache.kafka.clients.consumer.MockConsumer;
                        -import org.apache.kafka.clients.consumer.OffsetAndMetadata;
                        +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.Cluster;
                        @@ -57,10 +57,15 @@
                         import org.apache.kafka.streams.processor.TaskId;
                         import org.apache.kafka.streams.processor.TaskMetadata;
                         import org.apache.kafka.streams.processor.ThreadMetadata;
                        +import org.apache.kafka.streams.processor.internals.StreamThread.StreamsMetricsThreadImpl;
                         import org.apache.kafka.streams.processor.internals.testutil.LogCaptureAppender;
                         import org.apache.kafka.streams.state.KeyValueStore;
                        +import org.apache.kafka.streams.state.StoreBuilder;
                        +import org.apache.kafka.streams.state.internals.OffsetCheckpoint;
                         import org.apache.kafka.test.MockClientSupplier;
                        +import org.apache.kafka.test.MockProcessor;
                         import org.apache.kafka.test.MockStateRestoreListener;
                        +import org.apache.kafka.test.MockStoreBuilder;
                         import org.apache.kafka.test.MockTimestampExtractor;
                         import org.apache.kafka.test.TestCondition;
                         import org.apache.kafka.test.TestUtils;
                        @@ -68,7 +73,10 @@
                         import org.junit.Assert;
                         import org.junit.Before;
                         import org.junit.Test;
                        +import org.slf4j.Logger;
                         
                        +import java.io.File;
                        +import java.io.IOException;
                         import java.util.ArrayList;
                         import java.util.Collections;
                         import java.util.HashMap;
                        @@ -84,13 +92,16 @@
                         import static org.apache.kafka.common.utils.Utils.mkEntry;
                         import static org.apache.kafka.common.utils.Utils.mkMap;
                         import static org.apache.kafka.common.utils.Utils.mkProperties;
                        +import static org.apache.kafka.streams.processor.internals.AbstractStateManager.CHECKPOINT_FILE_NAME;
                         import static org.hamcrest.CoreMatchers.equalTo;
                        +import static org.hamcrest.CoreMatchers.not;
                        +import static org.hamcrest.CoreMatchers.nullValue;
                        +import static org.hamcrest.MatcherAssert.assertThat;
                         import static org.junit.Assert.assertEquals;
                         import static org.junit.Assert.assertFalse;
                         import static org.junit.Assert.assertNotEquals;
                         import static org.junit.Assert.assertNotNull;
                         import static org.junit.Assert.assertSame;
                        -import static org.junit.Assert.assertThat;
                         import static org.junit.Assert.assertTrue;
                         import static org.junit.Assert.fail;
                         
                        @@ -594,7 +605,7 @@ public void shouldOnlyShutdownOnce() {
                             @Test
                             public void shouldNotNullPointerWhenStandbyTasksAssignedAndNoStateStoresForTopology() {
                                 internalTopologyBuilder.addSource(null, "name", null, null, null, "topic");
                        -        internalTopologyBuilder.addSink("out", "output", null, null, null);
                        +        internalTopologyBuilder.addSink("out", "output", null, null, null, "name");
                         
                                 final StreamThread thread = createStreamThread(clientId, config, false);
                         
                        @@ -660,7 +671,7 @@ public void shouldCloseTaskAsZombieAndRemoveFromActiveTasksIfProducerWasFencedWh
                                     new TestCondition() {
                                         @Override
                                         public boolean conditionMet() {
                        -                    return producer.commitCount() == 1;
                        +                    return producer.commitCount() == 2;
                                         }
                                     },
                                     "StreamsThread did not commit transaction.");
                        @@ -681,13 +692,12 @@ public boolean conditionMet() {
                                     },
                                     "StreamsThread did not remove fenced zombie task.");
                         
                        -        assertThat(producer.commitCount(), equalTo(1L));
                        +        assertThat(producer.commitCount(), equalTo(2L));
                             }
                         
                        -    @Test
                        -    public void shouldCloseTaskAsZombieAndRemoveFromActiveTasksIfProducerGotFencedAtBeginTransactionWhenTaskIsResumed() {
                        +    private StreamThread setupStreamThread() {
                                 internalTopologyBuilder.addSource(null, "name", null, null, null, topic1);
                        -        internalTopologyBuilder.addSink("out", "output", null, null, null);
                        +        internalTopologyBuilder.addSink("out", "output", null, null, null, "name");
                         
                                 final StreamThread thread = createStreamThread(clientId, new StreamsConfig(configProps(true)), true);
                         
                        @@ -711,15 +721,32 @@ public void shouldCloseTaskAsZombieAndRemoveFromActiveTasksIfProducerGotFencedAt
                                 thread.runOnce(-1);
                         
                                 assertThat(thread.tasks().size(), equalTo(1));
                        +        return thread;
                        +    }
                        +
                        +    @Test
                        +    public void shouldCloseTaskAsZombieAndRemoveFromActiveTasksIfProducerGotFencedInCommitTransactionWhenSuspendingTaks() {
                        +        final StreamThread thread = setupStreamThread();
                         
                        -        thread.rebalanceListener.onPartitionsRevoked(null);
                                 clientSupplier.producers.get(0).fenceProducer();
                        -        thread.rebalanceListener.onPartitionsAssigned(assignedPartitions);
                        -        try {
                        -            thread.runOnce(-1);
                        -            fail("Should have thrown TaskMigratedException");
                        -        } catch (final TaskMigratedException expected) { /* ignore */ }
                        +        thread.rebalanceListener.onPartitionsRevoked(null);
                         
                        +        assertTrue(clientSupplier.producers.get(0).transactionInFlight());
                        +        assertFalse(clientSupplier.producers.get(0).transactionCommitted());
                        +        assertTrue(clientSupplier.producers.get(0).closed());
                        +        assertTrue(thread.tasks().isEmpty());
                        +    }
                        +
                        +    @Test
                        +    public void shouldCloseTaskAsZombieAndRemoveFromActiveTasksIfProducerGotFencedInCloseTransactionWhenSuspendingTaks() {
                        +        final StreamThread thread = setupStreamThread();
                        +
                        +        clientSupplier.producers.get(0).fenceProducerOnClose();
                        +        thread.rebalanceListener.onPartitionsRevoked(null);
                        +
                        +        assertFalse(clientSupplier.producers.get(0).transactionInFlight());
                        +        assertTrue(clientSupplier.producers.get(0).transactionCommitted());
                        +        assertFalse(clientSupplier.producers.get(0).closed());
                                 assertTrue(thread.tasks().isEmpty());
                             }
                         
                        @@ -821,12 +848,13 @@ public void shouldReturnStandbyTaskMetadataWhileRunningState() {
                         
                             @SuppressWarnings("unchecked")
                             @Test
                        -    public void shouldUpdateStandbyTask() {
                        +    public void shouldUpdateStandbyTask() throws IOException {
                                 final String storeName1 = "count-one";
                                 final String storeName2 = "table-two";
                        -        final String changelogName = applicationId + "-" + storeName1 + "-changelog";
                        -        final TopicPartition partition1 = new TopicPartition(changelogName, 1);
                        -        final TopicPartition partition2 = t2p1;
                        +        final String changelogName1 = applicationId + "-" + storeName1 + "-changelog";
                        +        final String changelogName2 = applicationId + "-" + storeName2 + "-changelog";
                        +        final TopicPartition partition1 = new TopicPartition(changelogName1, 1);
                        +        final TopicPartition partition2 = new TopicPartition(changelogName2, 1);
                                 internalStreamsBuilder.stream(Collections.singleton(topic1), consumed)
                                     .groupByKey().count(Materialized.>as(storeName1));
                                 final MaterializedInternal materialized = new MaterializedInternal(Materialized.as(storeName2));
                        @@ -835,10 +863,10 @@ public void shouldUpdateStandbyTask() {
                         
                                 final StreamThread thread = createStreamThread(clientId, config, false);
                                 final MockConsumer restoreConsumer = clientSupplier.restoreConsumer;
                        -        restoreConsumer.updatePartitions(changelogName,
                        +        restoreConsumer.updatePartitions(changelogName1,
                                     singletonList(
                                         new PartitionInfo(
                        -                    changelogName,
                        +                    changelogName1,
                                             1,
                                             null,
                                             new Node[0],
                        @@ -852,13 +880,13 @@ public void shouldUpdateStandbyTask() {
                                 restoreConsumer.updateBeginningOffsets(Collections.singletonMap(partition1, 0L));
                                 restoreConsumer.updateEndOffsets(Collections.singletonMap(partition2, 10L));
                                 restoreConsumer.updateBeginningOffsets(Collections.singletonMap(partition2, 0L));
                        -        // let the store1 be restored from 0 to 10; store2 be restored from 0 to (committed offset) 5
                        -        clientSupplier.consumer.assign(Utils.mkSet(partition2));
                        -        clientSupplier.consumer.commitSync(Collections.singletonMap(partition2, new OffsetAndMetadata(5L, "")));
                        +        // let the store1 be restored from 0 to 10; store2 be restored from 5 (checkpointed) to 10
                        +        OffsetCheckpoint checkpoint = new OffsetCheckpoint(new File(stateDirectory.directoryForTask(task3), CHECKPOINT_FILE_NAME));
                        +        checkpoint.write(Collections.singletonMap(partition2, 5L));
                         
                                 for (long i = 0L; i < 10L; i++) {
                        -            restoreConsumer.addRecord(new ConsumerRecord<>(changelogName, 1, i, ("K" + i).getBytes(), ("V" + i).getBytes()));
                        -            restoreConsumer.addRecord(new ConsumerRecord<>(topic2, 1, i, ("K" + i).getBytes(), ("V" + i).getBytes()));
                        +            restoreConsumer.addRecord(new ConsumerRecord<>(changelogName1, 1, i, ("K" + i).getBytes(), ("V" + i).getBytes()));
                        +            restoreConsumer.addRecord(new ConsumerRecord<>(changelogName2, 1, i, ("K" + i).getBytes(), ("V" + i).getBytes()));
                                 }
                         
                                 thread.setState(StreamThread.State.RUNNING);
                        @@ -884,9 +912,62 @@ public void shouldUpdateStandbyTask() {
                         
                                 assertEquals(10L, store1.approximateNumEntries());
                                 assertEquals(5L, store2.approximateNumEntries());
                        -        assertEquals(Collections.singleton(partition2), restoreConsumer.paused());
                        -        assertEquals(1, thread.standbyRecords().size());
                        -        assertEquals(5, thread.standbyRecords().get(partition2).size());
                        +        assertEquals(0, thread.standbyRecords().size());
                        +    }
                        +
                        +    @Test
                        +    public void shouldCreateStandbyTask() {
                        +        setupInternalTopologyWithoutState();
                        +        internalTopologyBuilder.addStateStore(new MockStoreBuilder("myStore", true), "processor1");
                        +
                        +        final StandbyTask standbyTask = createStandbyTask();
                        +
                        +        assertThat(standbyTask, not(nullValue()));
                        +    }
                        +
                        +    @Test
                        +    public void shouldNotCreateStandbyTaskWithoutStateStores() {
                        +        setupInternalTopologyWithoutState();
                        +
                        +        final StandbyTask standbyTask = createStandbyTask();
                        +
                        +        assertThat(standbyTask, nullValue());
                        +    }
                        +
                        +    @Test
                        +    public void shouldNotCreateStandbyTaskIfStateStoresHaveLoggingDisabled() {
                        +        setupInternalTopologyWithoutState();
                        +        final StoreBuilder storeBuilder = new MockStoreBuilder("myStore", true);
                        +        storeBuilder.withLoggingDisabled();
                        +        internalTopologyBuilder.addStateStore(storeBuilder, "processor1");
                        +
                        +        final StandbyTask standbyTask = createStandbyTask();
                        +
                        +        assertThat(standbyTask, nullValue());
                        +    }
                        +
                        +    private void setupInternalTopologyWithoutState() {
                        +        final MockProcessor mockProcessor = new MockProcessor();
                        +        internalTopologyBuilder.addSource(null, "source1", null, null, null, topic1);
                        +        internalTopologyBuilder.addProcessor("processor1", () -> mockProcessor, "source1");
                        +    }
                        +
                        +    private StandbyTask createStandbyTask() {
                        +        final LogContext logContext = new LogContext("test");
                        +        final Logger log = logContext.logger(StreamThreadTest.class);
                        +        final StreamsMetricsThreadImpl streamsMetrics = new StreamsMetricsThreadImpl(metrics, clientId);
                        +        final StreamThread.StandbyTaskCreator standbyTaskCreator = new StreamThread.StandbyTaskCreator(
                        +            internalTopologyBuilder,
                        +            config,
                        +            streamsMetrics,
                        +            stateDirectory,
                        +            new MockChangelogReader(),
                        +            mockTime,
                        +            log);
                        +        return standbyTaskCreator.createTask(
                        +            new MockConsumer<>(OffsetResetStrategy.EARLIEST),
                        +            new TaskId(1, 2),
                        +            Collections.emptySet());
                             }
                         
                             @Test
                        @@ -1103,7 +1184,7 @@ public boolean conditionMet() {
                                         }
                                     }, "Never restore first record");
                         
                        -            mockRestoreConsumer.setException(new InvalidOffsetException("Try Again!") {
                        +            mockRestoreConsumer.setPollException(new InvalidOffsetException("Try Again!") {
                                         @Override
                                         public Set partitions() {
                                             return changelogPartitionSet;
                        diff --git a/streams/src/test/java/org/apache/kafka/streams/processor/internals/StreamsPartitionAssignorTest.java b/streams/src/test/java/org/apache/kafka/streams/processor/internals/StreamsPartitionAssignorTest.java
                        index a32d193a171c2..4327e8f1ee49d 100644
                        --- a/streams/src/test/java/org/apache/kafka/streams/processor/internals/StreamsPartitionAssignorTest.java
                        +++ b/streams/src/test/java/org/apache/kafka/streams/processor/internals/StreamsPartitionAssignorTest.java
                        @@ -799,6 +799,7 @@ public Object apply(final Object value1, final Object value2) {
                                 final Map expectedCreatedInternalTopics = new HashMap<>();
                                 expectedCreatedInternalTopics.put(applicationId + "-KTABLE-AGGREGATE-STATE-STORE-0000000006-repartition", 4);
                                 expectedCreatedInternalTopics.put(applicationId + "-KTABLE-AGGREGATE-STATE-STORE-0000000006-changelog", 4);
                        +        expectedCreatedInternalTopics.put(applicationId + "-topic3-STATE-STORE-0000000002-changelog", 4);
                                 expectedCreatedInternalTopics.put(applicationId + "-KSTREAM-MAP-0000000001-repartition", 4);
                         
                                 // check if all internal topics were created as expected
                        diff --git a/streams/src/test/java/org/apache/kafka/streams/processor/internals/assignment/StickyTaskAssignorTest.java b/streams/src/test/java/org/apache/kafka/streams/processor/internals/assignment/StickyTaskAssignorTest.java
                        index ed22e3c30de84..d431dbeae2700 100644
                        --- a/streams/src/test/java/org/apache/kafka/streams/processor/internals/assignment/StickyTaskAssignorTest.java
                        +++ b/streams/src/test/java/org/apache/kafka/streams/processor/internals/assignment/StickyTaskAssignorTest.java
                        @@ -151,6 +151,25 @@ public void shouldAssignBasedOnCapacity() {
                                 assertThat(clients.get(p2).activeTasks().size(), equalTo(2));
                             }
                         
                        +    @Test
                        +    public void shouldAssignTasksEvenlyWithUnequalTopicGroupSizes() {
                        +
                        +        createClientWithPreviousActiveTasks(p1, 1, task00, task01, task02, task03,
                        +                                                            task04, task05, task10);
                        +
                        +        createClient(p2, 1);
                        +
                        +        final StickyTaskAssignor taskAssignor = createTaskAssignor(task10, task00, task01, task02, task03, task04, task05);
                        +
                        +        final Set expectedClientITasks = new HashSet<>(Arrays.asList(task00, task01, task10, task05));
                        +        final Set expectedClientIITasks = new HashSet<>(Arrays.asList(task02, task03, task04));
                        +
                        +        taskAssignor.assign(0);
                        +
                        +        assertThat(clients.get(p1).activeTasks(), equalTo(expectedClientITasks));
                        +        assertThat(clients.get(p2).activeTasks(), equalTo(expectedClientIITasks));
                        +    }
                        +
                             @Test
                             public void shouldKeepActiveTaskStickynessWhenMoreClientThanActiveTasks() {
                                 final int p5 = 5;
                        diff --git a/streams/src/test/java/org/apache/kafka/streams/processor/internals/StreamsMetricsImplTest.java b/streams/src/test/java/org/apache/kafka/streams/processor/internals/metrics/StreamsMetricsImplTest.java
                        similarity index 64%
                        rename from streams/src/test/java/org/apache/kafka/streams/processor/internals/StreamsMetricsImplTest.java
                        rename to streams/src/test/java/org/apache/kafka/streams/processor/internals/metrics/StreamsMetricsImplTest.java
                        index a72dc7928d70f..cccc458064480 100644
                        --- a/streams/src/test/java/org/apache/kafka/streams/processor/internals/StreamsMetricsImplTest.java
                        +++ b/streams/src/test/java/org/apache/kafka/streams/processor/internals/metrics/StreamsMetricsImplTest.java
                        @@ -14,14 +14,20 @@
                          * See the License for the specific language governing permissions and
                          * limitations under the License.
                          */
                        -package org.apache.kafka.streams.processor.internals;
                        +package org.apache.kafka.streams.processor.internals.metrics;
                         
                         
                        +import org.apache.kafka.common.MetricName;
                         import org.apache.kafka.common.metrics.Metrics;
                         import org.apache.kafka.common.metrics.Sensor;
                        -import org.apache.kafka.streams.processor.internals.metrics.StreamsMetricsImpl;
                        +import org.apache.kafka.common.metrics.stats.Count;
                         import org.junit.Test;
                         
                        +import java.util.Collections;
                        +import java.util.Map;
                        +
                        +import static org.apache.kafka.common.utils.Utils.mkEntry;
                        +import static org.apache.kafka.common.utils.Utils.mkMap;
                         import static org.junit.Assert.assertEquals;
                         
                         public class StreamsMetricsImplTest {
                        @@ -57,6 +63,55 @@ public void testRemoveSensor() {
                         
                                 final Sensor sensor3 = streamsMetrics.addThroughputSensor(taskName, scope, entity, operation, Sensor.RecordingLevel.DEBUG);
                                 streamsMetrics.removeSensor(sensor3);
                        +
                        +        assertEquals(Collections.emptyMap(), streamsMetrics.parentSensors());
                        +    }
                        +
                        +    @Test
                        +    public void testMutiLevelSensorRemoval() {
                        +        final Metrics registry = new Metrics();
                        +        final StreamsMetricsImpl metrics = new StreamsMetricsImpl(registry, "");
                        +        for (final MetricName defaultMetric : registry.metrics().keySet()) {
                        +            registry.removeMetric(defaultMetric);
                        +        }
                        +
                        +        final String taskName = "taskName";
                        +        final String operation = "operation";
                        +        final Map threadTags = mkMap(mkEntry("threadkey", "value"));
                        +
                        +        final Map taskTags = mkMap(mkEntry("taskkey", "value"));
                        +
                        +        final Sensor parent1 = metrics.threadLevelSensor(operation, Sensor.RecordingLevel.DEBUG);
                        +        parent1.add(new MetricName("name", "group", "description", threadTags), new Count());
                        +
                        +        assertEquals(1, registry.metrics().size());
                        +
                        +        final Sensor sensor1 = metrics.taskLevelSensor(taskName, operation, Sensor.RecordingLevel.DEBUG, parent1);
                        +        sensor1.add(new MetricName("name", "group", "description", taskTags), new Count());
                        +
                        +        assertEquals(2, registry.metrics().size());
                        +
                        +        metrics.removeAllTaskLevelSensors(taskName);
                        +
                        +        assertEquals(1, registry.metrics().size());
                        +
                        +        final Sensor parent2 = metrics.threadLevelSensor(operation, Sensor.RecordingLevel.DEBUG);
                        +        parent2.add(new MetricName("name", "group", "description", threadTags), new Count());
                        +
                        +        assertEquals(1, registry.metrics().size());
                        +
                        +        final Sensor sensor2 = metrics.taskLevelSensor(taskName, operation, Sensor.RecordingLevel.DEBUG, parent2);
                        +        sensor2.add(new MetricName("name", "group", "description", taskTags), new Count());
                        +
                        +        assertEquals(2, registry.metrics().size());
                        +
                        +        metrics.removeAllTaskLevelSensors(taskName);
                        +
                        +        assertEquals(1, registry.metrics().size());
                        +
                        +        metrics.removeAllThreadLevelSensors();
                        +
                        +        assertEquals(0, registry.metrics().size());
                             }
                         
                             @Test
                        @@ -90,7 +145,7 @@ public void testThroughputMetrics() {
                                 final String entity = "entity";
                                 final String operation = "put";
                         
                        -        final Sensor sensor1 = streamsMetrics.addThroughputSensor(taskName,  scope, entity, operation, Sensor.RecordingLevel.DEBUG);
                        +        final Sensor sensor1 = streamsMetrics.addThroughputSensor(taskName, scope, entity, operation, Sensor.RecordingLevel.DEBUG);
                         
                                 final int meterMetricsCount = 2; // Each Meter is a combination of a Rate and a Total
                                 // 2 meter metrics plus a common metric that keeps track of total registered metrics in Metrics() constructor
                        diff --git a/streams/src/test/java/org/apache/kafka/streams/processor/internals/testutil/LogCaptureAppender.java b/streams/src/test/java/org/apache/kafka/streams/processor/internals/testutil/LogCaptureAppender.java
                        index b6f5769149da9..462159f353416 100644
                        --- a/streams/src/test/java/org/apache/kafka/streams/processor/internals/testutil/LogCaptureAppender.java
                        +++ b/streams/src/test/java/org/apache/kafka/streams/processor/internals/testutil/LogCaptureAppender.java
                        @@ -21,11 +21,37 @@
                         import org.apache.log4j.Logger;
                         import org.apache.log4j.spi.LoggingEvent;
                         
                        +import java.util.Deque;
                         import java.util.LinkedList;
                         import java.util.List;
                        +import java.util.Optional;
                         
                         public class LogCaptureAppender extends AppenderSkeleton {
                        -    private final LinkedList events = new LinkedList<>();
                        +    private final Deque events = new LinkedList<>();
                        +
                        +    public static class Event {
                        +        private final String level;
                        +        private final String message;
                        +        private final Optional throwableInfo;
                        +
                        +        Event(final String level, final String message, final Optional throwableInfo) {
                        +            this.level = level;
                        +            this.message = message;
                        +            this.throwableInfo = throwableInfo;
                        +        }
                        +
                        +        public String getLevel() {
                        +            return level;
                        +        }
                        +
                        +        public String getMessage() {
                        +            return message;
                        +        }
                        +
                        +        public Optional getThrowableInfo() {
                        +            return throwableInfo;
                        +        }
                        +    }
                         
                             public static LogCaptureAppender createAndRegister() {
                                 final LogCaptureAppender logCaptureAppender = new LogCaptureAppender();
                        @@ -54,6 +80,30 @@ public List getMessages() {
                                 return result;
                             }
                         
                        +    public List getEvents() {
                        +        final LinkedList result = new LinkedList<>();
                        +        synchronized (events) {
                        +            for (final LoggingEvent event : events) {
                        +                final String[] throwableStrRep = event.getThrowableStrRep();
                        +                final Optional throwableString;
                        +                if (throwableStrRep == null) {
                        +                    throwableString = Optional.empty();
                        +                } else {
                        +                    final StringBuilder throwableStringBuilder = new StringBuilder();
                        +
                        +                    for (final String s : throwableStrRep) {
                        +                        throwableStringBuilder.append(s);
                        +                    }
                        +
                        +                    throwableString = Optional.of(throwableStringBuilder.toString());
                        +                }
                        +
                        +                result.add(new Event(event.getLevel().toString(), event.getRenderedMessage(), throwableString));
                        +            }
                        +        }
                        +        return result;
                        +    }
                        +
                             @Override
                             public void close() {
                         
                        diff --git a/streams/src/test/java/org/apache/kafka/streams/state/KeyValueStoreTestDriver.java b/streams/src/test/java/org/apache/kafka/streams/state/KeyValueStoreTestDriver.java
                        index 4b9e6a16eaf5c..3244e183f888a 100644
                        --- a/streams/src/test/java/org/apache/kafka/streams/state/KeyValueStoreTestDriver.java
                        +++ b/streams/src/test/java/org/apache/kafka/streams/state/KeyValueStoreTestDriver.java
                        @@ -189,7 +189,6 @@ private KeyValueStoreTestDriver(final StateSerdes serdes) {
                                 final Producer producer = new MockProducer<>(true, rawSerializer, rawSerializer);
                         
                                 final RecordCollector recordCollector = new RecordCollectorImpl(
                        -            producer,
                                     "KeyValueStoreTestDriver",
                                     new LogContext("KeyValueStoreTestDriver "),
                                     new DefaultProductionExceptionHandler(),
                        @@ -224,6 +223,7 @@ public  void send(final String topic,
                                         throw new UnsupportedOperationException();
                                     }
                                 };
                        +        recordCollector.init(producer);
                         
                                 final File stateDir = TestUtils.tempDirectory();
                                 //noinspection ResultOfMethodCallIgnored
                        diff --git a/streams/src/test/java/org/apache/kafka/streams/state/StateSerdesTest.java b/streams/src/test/java/org/apache/kafka/streams/state/StateSerdesTest.java
                        index 6f298886bc776..714ce18750043 100644
                        --- a/streams/src/test/java/org/apache/kafka/streams/state/StateSerdesTest.java
                        +++ b/streams/src/test/java/org/apache/kafka/streams/state/StateSerdesTest.java
                        @@ -18,6 +18,7 @@
                         
                         import org.apache.kafka.common.serialization.Serdes;
                         import org.apache.kafka.common.utils.Bytes;
                        +import org.apache.kafka.streams.errors.StreamsException;
                         import org.junit.Assert;
                         import org.junit.Test;
                         
                        @@ -86,4 +87,20 @@ public void shouldThrowIfValueClassIsNull() {
                                 new StateSerdes<>("anyName", Serdes.ByteArray(), null);
                             }
                         
                        +    @Test(expected = StreamsException.class)
                        +    public void shouldThrowIfIncompatibleSerdeForValue() throws ClassNotFoundException {
                        +        Class myClass = Class.forName("java.lang.String");
                        +        StateSerdes stateSerdes = new StateSerdes("anyName", Serdes.serdeFrom(myClass), Serdes.serdeFrom(myClass));
                        +        Integer myInt = 123;
                        +        stateSerdes.rawValue(myInt);
                        +    }
                        +
                        +    @Test(expected = StreamsException.class)
                        +    public void shouldThrowIfIncompatibleSerdeForKey() throws ClassNotFoundException {
                        +        Class myClass = Class.forName("java.lang.String");
                        +        StateSerdes stateSerdes = new StateSerdes("anyName", Serdes.serdeFrom(myClass), Serdes.serdeFrom(myClass));
                        +        Integer myInt = 123;
                        +        stateSerdes.rawKey(myInt);
                        +    }
                        +
                         }
                        diff --git a/streams/src/test/java/org/apache/kafka/streams/state/internals/CachingKeyValueStoreTest.java b/streams/src/test/java/org/apache/kafka/streams/state/internals/CachingKeyValueStoreTest.java
                        index 2f6aac79d0bca..8b9f95fb38e5b 100644
                        --- a/streams/src/test/java/org/apache/kafka/streams/state/internals/CachingKeyValueStoreTest.java
                        +++ b/streams/src/test/java/org/apache/kafka/streams/state/internals/CachingKeyValueStoreTest.java
                        @@ -34,6 +34,7 @@
                         import org.apache.kafka.streams.state.StoreBuilder;
                         import org.apache.kafka.streams.state.Stores;
                         import org.apache.kafka.test.InternalMockProcessorContext;
                        +import org.easymock.EasyMock;
                         import org.junit.After;
                         import org.junit.Before;
                         import org.junit.Test;
                        @@ -103,6 +104,22 @@ protected  KeyValueStore createKeyValueStore(final ProcessorContext
                                 return store;
                             }
                         
                        +    @Test
                        +    public void shouldCloseAfterErrorWithFlush() {
                        +        try {
                        +            cache = EasyMock.niceMock(ThreadCache.class);
                        +            context = new InternalMockProcessorContext(null, null, null, (RecordCollector) null, cache);
                        +            context.setRecordContext(new ProcessorRecordContext(10, 0, 0, topic, null));
                        +            store.init(context, null);
                        +            cache.flush("0_0-store");
                        +            EasyMock.expectLastCall().andThrow(new NullPointerException("Simulating an error on flush"));
                        +            EasyMock.replay(cache);
                        +            store.close();
                        +        } catch (final NullPointerException npe) {
                        +            assertFalse(underlyingStore.isOpen());
                        +        }
                        +    }
                        +
                             @Test
                             public void shouldPutGetToFromCache() {
                                 store.put(bytesKey("key"), bytesValue("value"));
                        @@ -274,7 +291,8 @@ public void shouldThrowNullPointerExceptionOnPutAllWithNullKey() {
                                 try {
                                     store.putAll(entries);
                                     fail("Should have thrown NullPointerException while putAll null key");
                        -        } catch (NullPointerException e) { }
                        +        } catch (final NullPointerException e) {
                        +        }
                             }
                         
                             @Test
                        @@ -326,4 +344,4 @@ public void apply(final K key, final V newValue, final V oldValue) {
                                     forwarded.put(key, new Change<>(newValue, oldValue));
                                 }
                             }
                        -}
                        \ No newline at end of file
                        +}
                        diff --git a/streams/src/test/java/org/apache/kafka/streams/state/internals/CachingWindowStoreTest.java b/streams/src/test/java/org/apache/kafka/streams/state/internals/CachingWindowStoreTest.java
                        index b8808caf475fd..551aeb12fc861 100644
                        --- a/streams/src/test/java/org/apache/kafka/streams/state/internals/CachingWindowStoreTest.java
                        +++ b/streams/src/test/java/org/apache/kafka/streams/state/internals/CachingWindowStoreTest.java
                        @@ -16,20 +16,31 @@
                          */
                         package org.apache.kafka.streams.state.internals;
                         
                        +import org.apache.kafka.clients.consumer.ConsumerConfig;
                         import org.apache.kafka.common.metrics.Metrics;
                         import org.apache.kafka.common.serialization.Serdes;
                         import org.apache.kafka.common.utils.Bytes;
                         import org.apache.kafka.common.utils.LogContext;
                         import org.apache.kafka.common.utils.Utils;
                         import org.apache.kafka.streams.KeyValue;
                        +import org.apache.kafka.streams.StreamsBuilder;
                        +import org.apache.kafka.streams.StreamsConfig;
                        +import org.apache.kafka.streams.TopologyTestDriver;
                         import org.apache.kafka.streams.errors.InvalidStateStoreException;
                        +import org.apache.kafka.streams.kstream.Consumed;
                        +import org.apache.kafka.streams.kstream.Transformer;
                         import org.apache.kafka.streams.kstream.Windowed;
                         import org.apache.kafka.streams.kstream.internals.TimeWindow;
                        +import org.apache.kafka.streams.processor.ProcessorContext;
                         import org.apache.kafka.streams.processor.internals.MockStreamsMetrics;
                         import org.apache.kafka.streams.processor.internals.ProcessorRecordContext;
                         import org.apache.kafka.streams.processor.internals.RecordCollector;
                         import org.apache.kafka.streams.state.KeyValueIterator;
                        +import org.apache.kafka.streams.state.StoreBuilder;
                        +import org.apache.kafka.streams.state.Stores;
                        +import org.apache.kafka.streams.state.WindowStore;
                         import org.apache.kafka.streams.state.WindowStoreIterator;
                        +import org.apache.kafka.streams.test.ConsumerRecordFactory;
                         import org.apache.kafka.test.InternalMockProcessorContext;
                         import org.apache.kafka.test.TestUtils;
                         import org.junit.After;
                        @@ -38,6 +49,8 @@
                         
                         import java.io.IOException;
                         import java.util.List;
                        +import java.util.Properties;
                        +import java.util.UUID;
                         
                         import static org.apache.kafka.streams.state.internals.ThreadCacheTest.memoryCacheEntrySize;
                         import static org.apache.kafka.test.StreamsTestUtils.toList;
                        @@ -90,6 +103,97 @@ public void closeStore() {
                                 cachingStore.close();
                             }
                         
                        +    @Test
                        +    public void shouldNotReturnDuplicatesInRanges() {
                        +        final StreamsBuilder builder = new StreamsBuilder();
                        +
                        +        final StoreBuilder> storeBuilder = Stores.windowStoreBuilder(
                        +            Stores.persistentWindowStore("store-name", 3600000L, 3, 60000L, false),
                        +            Serdes.String(),
                        +            Serdes.String())
                        +            .withCachingEnabled();
                        +
                        +        builder.addStateStore(storeBuilder);
                        +
                        +        builder.stream(topic,
                        +            Consumed.with(Serdes.String(), Serdes.String()))
                        +            .transform(() -> new Transformer>() {
                        +                private WindowStore store;
                        +                private int numRecordsProcessed;
                        +
                        +                @Override
                        +                public void init(final ProcessorContext processorContext) {
                        +                    this.store = (WindowStore) processorContext.getStateStore("store-name");
                        +                    int count = 0;
                        +
                        +                    final KeyValueIterator, String> all = store.all();
                        +                    while (all.hasNext()) {
                        +                        count++;
                        +                        all.next();
                        +                    }
                        +
                        +                    assertThat(count, equalTo(0));
                        +                }
                        +
                        +                @Override
                        +                public KeyValue transform(final String key, final String value) {
                        +                    int count = 0;
                        +
                        +                    final KeyValueIterator, String> all = store.all();
                        +                    while (all.hasNext()) {
                        +                        count++;
                        +                        all.next();
                        +                    }
                        +                    assertThat(count, equalTo(numRecordsProcessed));
                        +
                        +                    store.put(value, value);
                        +
                        +                    numRecordsProcessed++;
                        +
                        +                    return new KeyValue<>(key, value);
                        +                }
                        +
                        +                @Override
                        +                public void close() {
                        +
                        +                }
                        +            }, "store-name");
                        +
                        +        final String bootstrapServers = "localhost:9092";
                        +        final Properties streamsConfiguration = new Properties();
                        +        streamsConfiguration.put(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG, "earliest");
                        +        streamsConfiguration.put(StreamsConfig.APPLICATION_ID_CONFIG, "test-app");
                        +        streamsConfiguration.put(StreamsConfig.BOOTSTRAP_SERVERS_CONFIG, bootstrapServers);
                        +        streamsConfiguration.put(StreamsConfig.DEFAULT_KEY_SERDE_CLASS_CONFIG, Serdes.String().getClass().getName());
                        +        streamsConfiguration.put(StreamsConfig.DEFAULT_VALUE_SERDE_CLASS_CONFIG, Serdes.String().getClass().getName());
                        +        streamsConfiguration.put(StreamsConfig.STATE_DIR_CONFIG, TestUtils.tempDirectory().getPath());
                        +        streamsConfiguration.put(StreamsConfig.COMMIT_INTERVAL_MS_CONFIG, 10 * 1000);
                        +
                        +        final long initialWallClockTime = 0L;
                        +        final TopologyTestDriver driver = new TopologyTestDriver(builder.build(), streamsConfiguration, initialWallClockTime);
                        +
                        +        final ConsumerRecordFactory recordFactory = new ConsumerRecordFactory<>(Serdes.String().serializer(), Serdes.String().serializer(), initialWallClockTime);
                        +
                        +        for (int i = 0; i < 5; i++) {
                        +            driver.pipeInput(recordFactory.create(topic, UUID.randomUUID().toString(), UUID.randomUUID().toString()));
                        +        }
                        +        driver.advanceWallClockTime(10 * 1000L);
                        +        recordFactory.advanceTimeMs(10 * 1000L);
                        +        for (int i = 0; i < 5; i++) {
                        +            driver.pipeInput(recordFactory.create(topic, UUID.randomUUID().toString(), UUID.randomUUID().toString()));
                        +        }
                        +        driver.advanceWallClockTime(10 * 1000L);
                        +        recordFactory.advanceTimeMs(10 * 1000L);
                        +        for (int i = 0; i < 5; i++) {
                        +            driver.pipeInput(recordFactory.create(topic, UUID.randomUUID().toString(), UUID.randomUUID().toString()));
                        +        }
                        +        driver.advanceWallClockTime(10 * 1000L);
                        +        recordFactory.advanceTimeMs(10 * 1000L);
                        +        for (int i = 0; i < 5; i++) {
                        +            driver.pipeInput(recordFactory.create(topic, UUID.randomUUID().toString(), UUID.randomUUID().toString()));
                        +        }
                        +    }
                        +
                             @Test
                             public void shouldPutFetchFromCache() {
                                 cachingStore.put(bytesKey("a"), bytesValue("a"));
                        diff --git a/streams/src/test/java/org/apache/kafka/streams/state/internals/RocksDBWindowStoreTest.java b/streams/src/test/java/org/apache/kafka/streams/state/internals/RocksDBWindowStoreTest.java
                        index c436e9e588f1c..c2a1049382213 100644
                        --- a/streams/src/test/java/org/apache/kafka/streams/state/internals/RocksDBWindowStoreTest.java
                        +++ b/streams/src/test/java/org/apache/kafka/streams/state/internals/RocksDBWindowStoreTest.java
                        @@ -41,6 +41,7 @@
                         import org.apache.kafka.test.StreamsTestUtils;
                         import org.apache.kafka.test.TestUtils;
                         import org.junit.After;
                        +import org.junit.Before;
                         import org.junit.Test;
                         
                         import java.io.File;
                        @@ -79,7 +80,6 @@ public class RocksDBWindowStoreTest {
                         
                             private final Producer producer = new MockProducer<>(true, Serdes.ByteArray().serializer(), Serdes.ByteArray().serializer());
                             private final RecordCollector recordCollector = new RecordCollectorImpl(
                        -        producer,
                                 "RocksDBWindowStoreTestTask",
                                 new LogContext("RocksDBWindowStoreTestTask "),
                                 new DefaultProductionExceptionHandler(),
                        @@ -124,6 +124,11 @@ private WindowStore createWindowStore(final ProcessorContext co
                                 return createWindowStore(context, false);
                             }
                         
                        +    @Before
                        +    public void initRecordCollector() {
                        +        recordCollector.init(producer);
                        +    }
                        +
                             @After
                             public void closeStore() {
                                 if (windowStore != null) {
                        diff --git a/streams/src/test/java/org/apache/kafka/streams/state/internals/StoreChangeLoggerTest.java b/streams/src/test/java/org/apache/kafka/streams/state/internals/StoreChangeLoggerTest.java
                        index 5afe14f8a0a4e..7186c28b0fa46 100644
                        --- a/streams/src/test/java/org/apache/kafka/streams/state/internals/StoreChangeLoggerTest.java
                        +++ b/streams/src/test/java/org/apache/kafka/streams/state/internals/StoreChangeLoggerTest.java
                        @@ -43,7 +43,7 @@ public class StoreChangeLoggerTest {
                             private final Map loggedHeaders = new HashMap<>();
                         
                             private final InternalMockProcessorContext context = new InternalMockProcessorContext(StateSerdes.withBuiltinTypes(topic, Integer.class, String.class),
                        -        new RecordCollectorImpl(null, "StoreChangeLoggerTest", new LogContext("StoreChangeLoggerTest "), new DefaultProductionExceptionHandler(), new Metrics().sensor("skipped-records")) {
                        +        new RecordCollectorImpl("StoreChangeLoggerTest", new LogContext("StoreChangeLoggerTest "), new DefaultProductionExceptionHandler(), new Metrics().sensor("skipped-records")) {
                                     @Override
                                     public  void send(final String topic,
                                                               final K1 key,
                        diff --git a/streams/src/test/java/org/apache/kafka/streams/state/internals/StreamThreadStateStoreProviderTest.java b/streams/src/test/java/org/apache/kafka/streams/state/internals/StreamThreadStateStoreProviderTest.java
                        index c24122abd13d1..0780fba1a2a54 100644
                        --- a/streams/src/test/java/org/apache/kafka/streams/state/internals/StreamThreadStateStoreProviderTest.java
                        +++ b/streams/src/test/java/org/apache/kafka/streams/state/internals/StreamThreadStateStoreProviderTest.java
                        @@ -49,6 +49,7 @@
                         
                         import java.io.File;
                         import java.io.IOException;
                        +import java.time.Duration;
                         import java.util.Arrays;
                         import java.util.Collections;
                         import java.util.HashMap;
                        @@ -117,7 +118,7 @@ public void cleanUp() throws IOException {
                             public void shouldFindKeyValueStores() {
                                 mockThread(true);
                                 final List> kvStores =
                        -            provider.stores("kv-store", QueryableStoreTypes.keyValueStore());
                        +            provider.stores("kv-store", QueryableStoreTypes.keyValueStore());
                                 assertEquals(2, kvStores.size());
                             }
                         
                        @@ -177,13 +178,13 @@ private StreamTask createStreamsTask(final StreamsConfig streamsConfig,
                                     Collections.singletonList(new TopicPartition(topicName, taskId.partition)),
                                     topology,
                                     clientSupplier.consumer,
                        -            new StoreChangelogReader(clientSupplier.restoreConsumer, new MockStateRestoreListener(), new LogContext("test-stream-task ")),
                        +            new StoreChangelogReader(clientSupplier.restoreConsumer, Duration.ZERO, new MockStateRestoreListener(), new LogContext("test-stream-task ")),
                                     streamsConfig,
                                     new MockStreamsMetrics(metrics),
                                     stateDirectory,
                                     null,
                                     new MockTime(),
                        -            clientSupplier.getProducer(new HashMap())
                        +            () -> clientSupplier.getProducer(new HashMap<>())
                                 ) {
                                     @Override
                                     protected void updateOffsetLimits() {}
                        diff --git a/streams/src/test/java/org/apache/kafka/streams/tests/BrokerCompatibilityTest.java b/streams/src/test/java/org/apache/kafka/streams/tests/BrokerCompatibilityTest.java
                        index e897088beca05..3c8446ca46652 100644
                        --- a/streams/src/test/java/org/apache/kafka/streams/tests/BrokerCompatibilityTest.java
                        +++ b/streams/src/test/java/org/apache/kafka/streams/tests/BrokerCompatibilityTest.java
                        @@ -37,6 +37,7 @@
                         import org.apache.kafka.streams.kstream.ValueMapper;
                         
                         import java.io.IOException;
                        +import java.time.Duration;
                         import java.util.Collections;
                         import java.util.Locale;
                         import java.util.Properties;
                        @@ -153,7 +154,7 @@ private static void loopUntilRecordReceived(final String kafka, final boolean eo
                                     consumer.subscribe(Collections.singletonList(SINK_TOPIC));
                         
                                     while (true) {
                        -                final ConsumerRecords records = consumer.poll(100);
                        +                final ConsumerRecords records = consumer.poll(Duration.ofMillis(100));
                                         for (final ConsumerRecord record : records) {
                                             if (record.key().equals("key") && record.value().equals("1")) {
                                                 return;
                        diff --git a/streams/src/test/java/org/apache/kafka/streams/tests/EosTestDriver.java b/streams/src/test/java/org/apache/kafka/streams/tests/EosTestDriver.java
                        index 752cdd696eddd..0b18864d4ab2f 100644
                        --- a/streams/src/test/java/org/apache/kafka/streams/tests/EosTestDriver.java
                        +++ b/streams/src/test/java/org/apache/kafka/streams/tests/EosTestDriver.java
                        @@ -16,16 +16,18 @@
                          */
                         package org.apache.kafka.streams.tests;
                         
                        -import kafka.admin.AdminClient;
                        +import org.apache.kafka.clients.admin.AdminClient;
                        +import org.apache.kafka.clients.admin.ConsumerGroupDescription;
                        +import org.apache.kafka.clients.admin.KafkaAdminClient;
                        +import org.apache.kafka.clients.admin.ListConsumerGroupOffsetsResult;
                         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.KafkaConsumer;
                        -import org.apache.kafka.clients.producer.Callback;
                        +import org.apache.kafka.clients.consumer.OffsetAndMetadata;
                         import org.apache.kafka.clients.producer.KafkaProducer;
                         import org.apache.kafka.clients.producer.ProducerConfig;
                         import org.apache.kafka.clients.producer.ProducerRecord;
                        -import org.apache.kafka.clients.producer.RecordMetadata;
                         import org.apache.kafka.common.PartitionInfo;
                         import org.apache.kafka.common.TopicPartition;
                         import org.apache.kafka.common.errors.SerializationException;
                        @@ -40,17 +42,18 @@
                         import org.apache.kafka.common.utils.Exit;
                         import org.apache.kafka.common.utils.Utils;
                         
                        +import java.time.Duration;
                         import java.util.ArrayList;
                         import java.util.Collections;
                         import java.util.HashMap;
                        -import java.util.HashSet;
                         import java.util.Iterator;
                         import java.util.List;
                         import java.util.Locale;
                         import java.util.Map;
                         import java.util.Properties;
                         import java.util.Random;
                        -import java.util.Set;
                        +import java.util.concurrent.ExecutionException;
                        +import java.util.concurrent.TimeUnit;
                         
                         public class EosTestDriver extends SmokeTestUtil {
                         
                        @@ -59,22 +62,19 @@ public class EosTestDriver extends SmokeTestUtil {
                         
                             private static boolean isRunning = true;
                         
                        -    static int numRecordsProduced = 0;
                        +    private static int numRecordsProduced = 0;
                         
                        -    static synchronized void updateNumRecordsProduces(final int delta) {
                        +    private static synchronized void updateNumRecordsProduces(final int delta) {
                                 numRecordsProduced += delta;
                             }
                         
                             static void generate(final String kafka) {
                         
                        -        Runtime.getRuntime().addShutdownHook(new Thread() {
                        -            @Override
                        -            public void run() {
                        -                System.out.println("Terminating");
                        -                System.out.flush();
                        -                isRunning = false;
                        -            }
                        -        });
                        +        Runtime.getRuntime().addShutdownHook(new Thread(() -> {
                        +            System.out.println("Terminating");
                        +            System.out.flush();
                        +            isRunning = false;
                        +        }));
                         
                                 final Properties producerProps = new Properties();
                                 producerProps.put(ProducerConfig.CLIENT_ID_CONFIG, "EosTest");
                        @@ -93,19 +93,16 @@ public void run() {
                         
                                     final ProducerRecord record = new ProducerRecord<>("data", key, value);
                         
                        -            producer.send(record, new Callback() {
                        -                @Override
                        -                public void onCompletion(final RecordMetadata metadata, final Exception exception) {
                        -                    if (exception != null) {
                        -                        exception.printStackTrace(System.err);
                        -                        System.err.flush();
                        -                        if (exception instanceof TimeoutException) {
                        -                            try {
                        -                                // message == org.apache.kafka.common.errors.TimeoutException: Expiring 4 record(s) for data-0: 30004 ms has passed since last attempt plus backoff time
                        -                                final int expired = Integer.parseInt(exception.getMessage().split(" ")[2]);
                        -                                updateNumRecordsProduces(-expired);
                        -                            } catch (Exception ignore) { }
                        -                        }
                        +            producer.send(record, (metadata, exception) -> {
                        +                if (exception != null) {
                        +                    exception.printStackTrace(System.err);
                        +                    System.err.flush();
                        +                    if (exception instanceof TimeoutException) {
                        +                        try {
                        +                            // message == org.apache.kafka.common.errors.TimeoutException: Expiring 4 record(s) for data-0: 30004 ms has passed since last attempt plus backoff time
                        +                            final int expired = Integer.parseInt(exception.getMessage().split(" ")[2]);
                        +                            updateNumRecordsProduces(-expired);
                        +                        } catch (final Exception ignore) { }
                                             }
                                         }
                                     });
                        @@ -141,10 +138,6 @@ public void onCompletion(final RecordMetadata metadata, final Exception exceptio
                             }
                         
                             public static void verify(final String kafka, final boolean withRepartitioning) {
                        -        ensureStreamsApplicationDown(kafka);
                        -
                        -        final Map committedOffsets = getCommittedOffsets(kafka, withRepartitioning);
                        -
                                 final Properties props = new Properties();
                                 props.put(ConsumerConfig.CLIENT_ID_CONFIG, "verifier");
                                 props.put(ConsumerConfig.BOOTSTRAP_SERVERS_CONFIG, kafka);
                        @@ -152,6 +145,13 @@ public static void verify(final String kafka, final boolean withRepartitioning)
                                 props.put(ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG, ByteArrayDeserializer.class);
                                 props.put(ConsumerConfig.ISOLATION_LEVEL_CONFIG, IsolationLevel.READ_COMMITTED.toString().toLowerCase(Locale.ROOT));
                         
                        +        final Map committedOffsets;
                        +        try (final AdminClient adminClient = KafkaAdminClient.create(props)) {
                        +            ensureStreamsApplicationDown(adminClient);
                        +
                        +            committedOffsets = getCommittedOffsets(adminClient, withRepartitioning);
                        +        }
                        +
                                 final String[] allInputTopics;
                                 final String[] allOutputTopics;
                                 if (withRepartitioning) {
                        @@ -218,54 +218,42 @@ public static void verify(final String kafka, final boolean withRepartitioning)
                                 System.out.flush();
                             }
                         
                        -    private static void ensureStreamsApplicationDown(final String kafka) {
                        -        AdminClient adminClient = null;
                        -        try {
                        -            adminClient = AdminClient.createSimplePlaintext(kafka);
                        +    private static void ensureStreamsApplicationDown(final AdminClient adminClient) {
                         
                        -            final long maxWaitTime = System.currentTimeMillis() + MAX_IDLE_TIME_MS;
                        -            while (!adminClient.describeConsumerGroup(EosTestClient.APP_ID, 10000).consumers().get().isEmpty()) {
                        -                if (System.currentTimeMillis() > maxWaitTime) {
                        -                    throw new RuntimeException("Streams application not down after " + (MAX_IDLE_TIME_MS / 1000) + " seconds.");
                        -                }
                        -                sleep(1000);
                        -            }
                        -        } finally {
                        -            if (adminClient != null) {
                        -                adminClient.close();
                        +        final long maxWaitTime = System.currentTimeMillis() + MAX_IDLE_TIME_MS;
                        +        ConsumerGroupDescription description;
                        +        do {
                        +            description = getConsumerGroupDescription(adminClient);
                        +
                        +            if (System.currentTimeMillis() > maxWaitTime && !description.members().isEmpty()) {
                        +                throw new RuntimeException(
                        +                    "Streams application not down after " + (MAX_IDLE_TIME_MS / 1000) + " seconds. " +
                        +                        "Group: " + description
                        +                );
                                     }
                        -        }
                        +            sleep(1000);
                        +        } while (!description.members().isEmpty());
                             }
                         
                        -    private static Map getCommittedOffsets(final String kafka,
                        +
                        +    private static Map getCommittedOffsets(final AdminClient adminClient,
                                                                                          final boolean withRepartitioning) {
                        -        final Properties props = new Properties();
                        -        props.put(ConsumerConfig.BOOTSTRAP_SERVERS_CONFIG, kafka);
                        -        props.put(ConsumerConfig.GROUP_ID_CONFIG, EosTestClient.APP_ID);
                        -        props.put(ConsumerConfig.CLIENT_ID_CONFIG, "OffsetsClient");
                        -        props.put(ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG, ByteArrayDeserializer.class);
                        -        props.put(ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG, ByteArrayDeserializer.class);
                        +        final Map topicPartitionOffsetAndMetadataMap;
                         
                        -        final Map committedOffsets = new HashMap<>();
                        -        try (final KafkaConsumer consumer = new KafkaConsumer<>(props)) {
                        -            final Set topics = new HashSet<>();
                        -            topics.add("data");
                        -            if (withRepartitioning) {
                        -                topics.add("repartition");
                        -            }
                        -            consumer.subscribe(topics);
                        -            consumer.poll(0);
                        +        try {
                        +            final ListConsumerGroupOffsetsResult listConsumerGroupOffsetsResult = adminClient.listConsumerGroupOffsets(EosTestClient.APP_ID);
                        +            topicPartitionOffsetAndMetadataMap = listConsumerGroupOffsetsResult.partitionsToOffsetAndMetadata().get(10, TimeUnit.SECONDS);
                        +        } catch (final InterruptedException | ExecutionException | java.util.concurrent.TimeoutException e) {
                        +            e.printStackTrace();
                        +            throw new RuntimeException(e);
                        +        }
                         
                        -            final Set partitions = new HashSet<>();
                        -            for (final String topic : topics) {
                        -                for (final PartitionInfo partition : consumer.partitionsFor(topic)) {
                        -                    partitions.add(new TopicPartition(partition.topic(), partition.partition()));
                        -                }
                        -            }
                        +        final Map committedOffsets = new HashMap<>();
                         
                        -            for (final TopicPartition tp : partitions) {
                        -                final long offset = consumer.position(tp);
                        -                committedOffsets.put(tp, offset);
                        +        for (final Map.Entry entry : topicPartitionOffsetAndMetadataMap.entrySet()) {
                        +            final String topic = entry.getKey().topic();
                        +            if (topic.equals("data") || withRepartitioning && topic.equals("repartition")) {
                        +                committedOffsets.put(entry.getKey(), entry.getValue().offset());
                                     }
                                 }
                         
                        @@ -284,7 +272,7 @@ private static Map receivedRecords = consumer.poll(100);
                        +            final ConsumerRecords receivedRecords = consumer.poll(Duration.ofMillis(100));
                         
                                     for (final ConsumerRecord record : receivedRecords) {
                                         maxWaitTime = System.currentTimeMillis() + MAX_IDLE_TIME_MS;
                        @@ -327,19 +315,12 @@ private static void addRecord(final ConsumerRecord record,
                                 final TopicPartition partition = new TopicPartition(topic, record.partition());
                         
                                 if (verifyTopic(topic, withRepartitioning)) {
                        -            Map>> topicRecordsPerPartition
                        -                = recordPerTopicPerPartition.get(topic);
                        +            final Map>> topicRecordsPerPartition =
                        +                recordPerTopicPerPartition.computeIfAbsent(topic, k -> new HashMap<>());
                         
                        -            if (topicRecordsPerPartition == null) {
                        -                topicRecordsPerPartition = new HashMap<>();
                        -                recordPerTopicPerPartition.put(topic, topicRecordsPerPartition);
                        -            }
                        +            final List> records =
                        +                topicRecordsPerPartition.computeIfAbsent(partition, k -> new ArrayList<>());
                         
                        -            List> records = topicRecordsPerPartition.get(partition);
                        -            if (records == null) {
                        -                records = new ArrayList<>();
                        -                topicRecordsPerPartition.put(partition, records);
                        -            }
                                     records.add(record);
                                 } else {
                                     throw new RuntimeException("FAIL: received data from unexpected topic: " + record);
                        @@ -397,7 +378,7 @@ private static void verifyMin(final Map> inputRecords = partitionInput.iterator();
                        @@ -439,7 +420,7 @@ private static void verifySum(final Map> inputRecords = partitionInput.iterator();
                        @@ -480,7 +461,7 @@ private static void verifyMax(final Map> inputRecords = partitionInput.iterator();
                        @@ -501,7 +482,7 @@ private static void verifyMax(final Map but was <" + receivedKey + "," + receivedValue + ">");
                                         }
                                     }
                        @@ -521,7 +502,7 @@ private static void verifyCnt(final Map> inputRecords = partitionInput.iterator();
                        @@ -539,7 +520,7 @@ private static void verifyCnt(final Map but was <" + receivedKey + "," + receivedValue + ">");
                                         }
                                     }
                        @@ -574,14 +555,11 @@ private static void verifyAllTransactionFinished(final KafkaConsumer record = new ProducerRecord<>(tp.topic(), tp.partition(), "key", "value");
                         
                        -                producer.send(record, new Callback() {
                        -                    @Override
                        -                    public void onCompletion(final RecordMetadata metadata, final Exception exception) {
                        -                        if (exception != null) {
                        -                            exception.printStackTrace(System.err);
                        -                            System.err.flush();
                        -                            Exit.exit(1);
                        -                        }
                        +                producer.send(record, (metadata, exception) -> {
                        +                    if (exception != null) {
                        +                        exception.printStackTrace(System.err);
                        +                        System.err.flush();
                        +                        Exit.exit(1);
                                             }
                                         });
                                     }
                        @@ -591,7 +569,7 @@ public void onCompletion(final RecordMetadata metadata, final Exception exceptio
                         
                                 long maxWaitTime = System.currentTimeMillis() + MAX_IDLE_TIME_MS;
                                 while (!partitions.isEmpty() && System.currentTimeMillis() < maxWaitTime) {
                        -            final ConsumerRecords records = consumer.poll(100);
                        +            final ConsumerRecords records = consumer.poll(Duration.ofMillis(100));
                                     if (records.isEmpty()) {
                                         System.out.println("No data received.");
                                         for (final TopicPartition tp : partitions) {
                        @@ -638,4 +616,18 @@ private static List getAllPartitions(final KafkaConsumer c
                                 return partitions;
                             }
                         
                        +
                        +    private static ConsumerGroupDescription getConsumerGroupDescription(final AdminClient adminClient) {
                        +        final ConsumerGroupDescription description;
                        +        try {
                        +            description = adminClient.describeConsumerGroups(Collections.singleton(EosTestClient.APP_ID))
                        +                .describedGroups()
                        +                .get(EosTestClient.APP_ID)
                        +                .get(10, TimeUnit.SECONDS);
                        +        } catch (final InterruptedException | ExecutionException | java.util.concurrent.TimeoutException e) {
                        +            e.printStackTrace();
                        +            throw new RuntimeException("Unexpected Exception getting group description", e);
                        +        }
                        +        return description;
                        +    }
                         }
                        diff --git a/streams/src/test/java/org/apache/kafka/streams/tests/SmokeTestDriver.java b/streams/src/test/java/org/apache/kafka/streams/tests/SmokeTestDriver.java
                        index 50330a08e612c..7533fdd085883 100644
                        --- a/streams/src/test/java/org/apache/kafka/streams/tests/SmokeTestDriver.java
                        +++ b/streams/src/test/java/org/apache/kafka/streams/tests/SmokeTestDriver.java
                        @@ -36,6 +36,7 @@
                         import org.apache.kafka.test.TestUtils;
                         
                         import java.io.File;
                        +import java.time.Duration;
                         import java.util.ArrayList;
                         import java.util.Collections;
                         import java.util.HashMap;
                        @@ -289,7 +290,7 @@ public static void verify(String kafka, Map> allData, int m
                                 int retry = 0;
                                 final long start = System.currentTimeMillis();
                                 while (System.currentTimeMillis() - start < TimeUnit.MINUTES.toMillis(6)) {
                        -            ConsumerRecords records = consumer.poll(500);
                        +            ConsumerRecords records = consumer.poll(Duration.ofMillis(500));
                                     if (records.isEmpty() && recordsProcessed >= recordsGenerated) {
                                         if (verifyMin(min, allData, false)
                                             && verifyMax(max, allData, false)
                        diff --git a/streams/src/test/java/org/apache/kafka/streams/tools/StreamsResetterTest.java b/streams/src/test/java/org/apache/kafka/streams/tools/StreamsResetterTest.java
                        index ad19f32fd1d74..33cf1fa34bcf1 100644
                        --- a/streams/src/test/java/org/apache/kafka/streams/tools/StreamsResetterTest.java
                        +++ b/streams/src/test/java/org/apache/kafka/streams/tools/StreamsResetterTest.java
                        @@ -32,6 +32,7 @@
                         
                         import java.text.ParseException;
                         import java.text.SimpleDateFormat;
                        +import java.time.Duration;
                         import java.util.Collections;
                         import java.util.Date;
                         import java.util.HashMap;
                        @@ -74,7 +75,7 @@ public void testResetToSpecificOffsetWhenBetweenBeginningAndEndOffset() {
                         
                                 streamsResetter.resetOffsetsTo(consumer, inputTopicPartitions, 2L);
                         
                        -        final ConsumerRecords records = consumer.poll(500);
                        +        final ConsumerRecords records = consumer.poll(Duration.ofMillis(500));
                                 assertEquals(3, records.count());
                             }
                         
                        @@ -90,7 +91,7 @@ public void testResetToSpecificOffsetWhenBeforeBeginningOffset() {
                         
                                 streamsResetter.resetOffsetsTo(consumer, inputTopicPartitions, 2L);
                         
                        -        final ConsumerRecords records = consumer.poll(500);
                        +        final ConsumerRecords records = consumer.poll(Duration.ofMillis(500));
                                 assertEquals(2, records.count());
                             }
                         
                        @@ -106,7 +107,7 @@ public void testResetToSpecificOffsetWhenAfterEndOffset() {
                         
                                 streamsResetter.resetOffsetsTo(consumer, inputTopicPartitions, 4L);
                         
                        -        final ConsumerRecords records = consumer.poll(500);
                        +        final ConsumerRecords records = consumer.poll(Duration.ofMillis(500));
                                 assertEquals(2, records.count());
                             }
                         
                        @@ -122,7 +123,7 @@ public void testShiftOffsetByWhenBetweenBeginningAndEndOffset() {
                         
                                 streamsResetter.shiftOffsetsBy(consumer, inputTopicPartitions, 3L);
                         
                        -        final ConsumerRecords records = consumer.poll(500);
                        +        final ConsumerRecords records = consumer.poll(Duration.ofMillis(500));
                                 assertEquals(2, records.count());
                             }
                         
                        @@ -138,7 +139,7 @@ public void testShiftOffsetByWhenBeforeBeginningOffset() {
                         
                                 streamsResetter.shiftOffsetsBy(consumer, inputTopicPartitions, -3L);
                         
                        -        final ConsumerRecords records = consumer.poll(500);
                        +        final ConsumerRecords records = consumer.poll(Duration.ofMillis(500));
                                 assertEquals(5, records.count());
                             }
                         
                        @@ -154,7 +155,7 @@ public void testShiftOffsetByWhenAfterEndOffset() {
                         
                                 streamsResetter.shiftOffsetsBy(consumer, inputTopicPartitions, 5L);
                         
                        -        final ConsumerRecords records = consumer.poll(500);
                        +        final ConsumerRecords records = consumer.poll(Duration.ofMillis(500));
                                 assertEquals(2, records.count());
                             }
                         
                        @@ -172,7 +173,7 @@ public void testResetUsingPlanWhenBetweenBeginningAndEndOffset() {
                                 topicPartitionsAndOffset.put(topicPartition, 3L);
                                 streamsResetter.resetOffsetsFromResetPlan(consumer, inputTopicPartitions, topicPartitionsAndOffset);
                         
                        -        final ConsumerRecords records = consumer.poll(500);
                        +        final ConsumerRecords records = consumer.poll(Duration.ofMillis(500));
                                 assertEquals(2, records.count());
                             }
                         
                        @@ -190,7 +191,7 @@ public void testResetUsingPlanWhenBeforeBeginningOffset() {
                                 topicPartitionsAndOffset.put(topicPartition, 1L);
                                 streamsResetter.resetOffsetsFromResetPlan(consumer, inputTopicPartitions, topicPartitionsAndOffset);
                         
                        -        final ConsumerRecords records = consumer.poll(500);
                        +        final ConsumerRecords records = consumer.poll(Duration.ofMillis(500));
                                 assertEquals(2, records.count());
                             }
                         
                        @@ -208,7 +209,7 @@ public void testResetUsingPlanWhenAfterEndOffset() {
                                 topicPartitionsAndOffset.put(topicPartition, 5L);
                                 streamsResetter.resetOffsetsFromResetPlan(consumer, inputTopicPartitions, topicPartitionsAndOffset);
                         
                        -        final ConsumerRecords records = consumer.poll(500);
                        +        final ConsumerRecords records = consumer.poll(Duration.ofMillis(500));
                                 assertEquals(2, records.count());
                             }
                         
                        @@ -226,7 +227,7 @@ public void shouldSeekToEndOffset() {
                                 intermediateTopicPartitions.add(topicPartition);
                                 streamsResetter.maybeSeekToEnd("g1", consumer, intermediateTopicPartitions);
                         
                        -        final ConsumerRecords records = consumer.poll(500);
                        +        final ConsumerRecords records = consumer.poll(Duration.ofMillis(500));
                                 assertEquals(2, records.count());
                             }
                         
                        diff --git a/streams/src/test/java/org/apache/kafka/test/KStreamTestDriver.java b/streams/src/test/java/org/apache/kafka/test/KStreamTestDriver.java
                        index 698cdc7ff85f3..b83936b8df512 100644
                        --- a/streams/src/test/java/org/apache/kafka/test/KStreamTestDriver.java
                        +++ b/streams/src/test/java/org/apache/kafka/test/KStreamTestDriver.java
                        @@ -235,7 +235,7 @@ private ProcessorRecordContext createRecordContext(final String topicName, final
                         
                             private class MockRecordCollector extends RecordCollectorImpl {
                                 MockRecordCollector() {
                        -            super(null, "KStreamTestDriver", new LogContext("KStreamTestDriver "), new DefaultProductionExceptionHandler(), new Metrics().sensor("skipped-records"));
                        +            super("KStreamTestDriver", new LogContext("KStreamTestDriver "), new DefaultProductionExceptionHandler(), new Metrics().sensor("skipped-records"));
                                 }
                         
                                 @Override
                        diff --git a/streams/src/test/java/org/apache/kafka/test/MockRestoreConsumer.java b/streams/src/test/java/org/apache/kafka/test/MockRestoreConsumer.java
                        index 3070e36482f21..00788fd2f98ce 100644
                        --- a/streams/src/test/java/org/apache/kafka/test/MockRestoreConsumer.java
                        +++ b/streams/src/test/java/org/apache/kafka/test/MockRestoreConsumer.java
                        @@ -25,6 +25,7 @@
                         import org.apache.kafka.common.serialization.IntegerSerializer;
                         import org.apache.kafka.common.serialization.Serializer;
                         
                        +import java.time.Duration;
                         import java.util.ArrayList;
                         import java.util.Collection;
                         import java.util.Collections;
                        @@ -85,9 +86,8 @@ public synchronized void assign(Collection partitions) {
                                 super.assign(partitions);
                             }
                         
                        -    @Deprecated
                             @Override
                        -    public ConsumerRecords poll(long timeout) {
                        +    public ConsumerRecords poll(final Duration timeout) {
                                 // add buffered records to MockConsumer
                                 for (ConsumerRecord record : recordBuffer) {
                                     super.addRecord(record);
                        diff --git a/streams/src/test/java/org/apache/kafka/test/NoOpReadOnlyStore.java b/streams/src/test/java/org/apache/kafka/test/NoOpReadOnlyStore.java
                        index ae46b8dadaa9b..08945d5047a04 100644
                        --- a/streams/src/test/java/org/apache/kafka/test/NoOpReadOnlyStore.java
                        +++ b/streams/src/test/java/org/apache/kafka/test/NoOpReadOnlyStore.java
                        @@ -95,7 +95,7 @@ public void close() {
                         
                             @Override
                             public boolean persistent() {
                        -        return false;
                        +        return rocksdbStore;
                             }
                         
                             @Override
                        diff --git a/streams/src/test/java/org/apache/kafka/test/NoOpRecordCollector.java b/streams/src/test/java/org/apache/kafka/test/NoOpRecordCollector.java
                        index 893d3566c6abc..07ba9b4b98ca1 100644
                        --- a/streams/src/test/java/org/apache/kafka/test/NoOpRecordCollector.java
                        +++ b/streams/src/test/java/org/apache/kafka/test/NoOpRecordCollector.java
                        @@ -16,6 +16,7 @@
                          */
                         package org.apache.kafka.test;
                         
                        +import org.apache.kafka.clients.producer.Producer;
                         import org.apache.kafka.common.TopicPartition;
                         import org.apache.kafka.common.header.Headers;
                         import org.apache.kafka.common.serialization.Serializer;
                        @@ -47,6 +48,9 @@ public  void send(final String topic,
                                                     final Serializer valueSerializer,
                                                     final StreamPartitioner partitioner) {}
                         
                        +    @Override
                        +    public void init(final Producer producer) {}
                        +
                             @Override
                             public void flush() {}
                         
                        diff --git a/streams/streams-scala/src/main/scala/org/apache/kafka/streams/scala/FunctionConversions.scala b/streams/streams-scala/src/main/scala/org/apache/kafka/streams/scala/FunctionConversions.scala
                        index 9ce9838c7c9a0..566ba22e615c8 100644
                        --- a/streams/streams-scala/src/main/scala/org/apache/kafka/streams/scala/FunctionConversions.scala
                        +++ b/streams/streams-scala/src/main/scala/org/apache/kafka/streams/scala/FunctionConversions.scala
                        @@ -30,17 +30,23 @@ import java.lang.{Iterable => JIterable}
                          * more expressive, with less boilerplate and more succinct.
                          * 

                        * For Scala 2.11, most of these conversions need to be invoked explicitly, as Scala 2.11 does not - * have full support for SAM types. + * have full support for SAM types. */ object FunctionConversions { + implicit private[scala] class ForeachActionFromFunction[K, V](val p: (K, V) => Unit) extends AnyVal { + def asForeachAction: ForeachAction[K, V] = new ForeachAction[K, V] { + override def apply(key: K, value: V): Unit = p(key, value) + } + } + implicit class PredicateFromFunction[K, V](val p: (K, V) => Boolean) extends AnyVal { def asPredicate: Predicate[K, V] = new Predicate[K, V] { override def test(key: K, value: V): Boolean = p(key, value) } } - implicit class MapperFromFunction[T, U, VR](val f:(T,U) => VR) extends AnyVal { + implicit class MapperFromFunction[T, U, VR](val f: (T, U) => VR) extends AnyVal { def asKeyValueMapper: KeyValueMapper[T, U, VR] = new KeyValueMapper[T, U, VR] { override def apply(key: T, value: U): VR = f(key, value) } @@ -49,7 +55,7 @@ object FunctionConversions { } } - implicit class KeyValueMapperFromFunction[K, V, KR, VR](val f:(K,V) => (KR, VR)) extends AnyVal { + implicit class KeyValueMapperFromFunction[K, V, KR, VR](val f: (K, V) => (KR, VR)) extends AnyVal { def asKeyValueMapper: KeyValueMapper[K, V, KeyValue[KR, VR]] = new KeyValueMapper[K, V, KeyValue[KR, VR]] { override def apply(key: K, value: V): KeyValue[KR, VR] = { val (kr, vr) = f(key, value) @@ -88,7 +94,7 @@ object FunctionConversions { } } - implicit class MergerFromFunction[K,VR](val f: (K, VR, VR) => VR) extends AnyVal { + implicit class MergerFromFunction[K, VR](val f: (K, VR, VR) => VR) extends AnyVal { def asMerger: Merger[K, VR] = new Merger[K, VR] { override def apply(aggKey: K, aggOne: VR, aggTwo: VR): VR = f(aggKey, aggOne, aggTwo) } @@ -105,4 +111,10 @@ object FunctionConversions { override def apply(): VA = f() } } + + implicit class TransformerSupplierFromFunction[K, V, VO](val f: () => Transformer[K, V, VO]) extends AnyVal { + def asTransformerSupplier: TransformerSupplier[K, V, VO] = new TransformerSupplier[K, V, VO] { + override def get(): Transformer[K, V, VO] = f() + } + } } diff --git a/streams/streams-scala/src/main/scala/org/apache/kafka/streams/scala/ImplicitConversions.scala b/streams/streams-scala/src/main/scala/org/apache/kafka/streams/scala/ImplicitConversions.scala index 0c384a1bad01c..d1ff6747d87e9 100644 --- a/streams/streams-scala/src/main/scala/org/apache/kafka/streams/scala/ImplicitConversions.scala +++ b/streams/streams-scala/src/main/scala/org/apache/kafka/streams/scala/ImplicitConversions.scala @@ -77,7 +77,8 @@ object ImplicitConversions { valueSerde: Serde[V]): Materialized[K, V, S] = Materialized.`with`[K, V, S](keySerde, valueSerde) - implicit def joinedFromKeyValueOtherSerde[K, V, VO] - (implicit keySerde: Serde[K], valueSerde: Serde[V], otherValueSerde: Serde[VO]): Joined[K, V, VO] = + implicit def joinedFromKeyValueOtherSerde[K, V, VO](implicit keySerde: Serde[K], + valueSerde: Serde[V], + otherValueSerde: Serde[VO]): Joined[K, V, VO] = Joined.`with`(keySerde, valueSerde, otherValueSerde) } diff --git a/streams/streams-scala/src/main/scala/org/apache/kafka/streams/scala/Serdes.scala b/streams/streams-scala/src/main/scala/org/apache/kafka/streams/scala/Serdes.scala index a0ffffaf66063..02e5380e71079 100644 --- a/streams/streams-scala/src/main/scala/org/apache/kafka/streams/scala/Serdes.scala +++ b/streams/streams-scala/src/main/scala/org/apache/kafka/streams/scala/Serdes.scala @@ -25,47 +25,48 @@ import org.apache.kafka.common.serialization.{Deserializer, Serde, Serializer, S import org.apache.kafka.streams.kstream.WindowedSerdes object Serdes { - implicit val String: Serde[String] = JSerdes.String() - implicit val Long: Serde[Long] = JSerdes.Long().asInstanceOf[Serde[Long]] - implicit val JavaLong: Serde[java.lang.Long] = JSerdes.Long() - implicit val ByteArray: Serde[Array[Byte]] = JSerdes.ByteArray() - implicit val Bytes: Serde[org.apache.kafka.common.utils.Bytes] = JSerdes.Bytes() - implicit val Float: Serde[Float] = JSerdes.Float().asInstanceOf[Serde[Float]] - implicit val JavaFloat: Serde[java.lang.Float] = JSerdes.Float() - implicit val Double: Serde[Double] = JSerdes.Double().asInstanceOf[Serde[Double]] - implicit val JavaDouble: Serde[java.lang.Double] = JSerdes.Double() - implicit val Integer: Serde[Int] = JSerdes.Integer().asInstanceOf[Serde[Int]] - implicit val JavaInteger: Serde[java.lang.Integer] = JSerdes.Integer() + implicit def String: Serde[String] = JSerdes.String() + implicit def Long: Serde[Long] = JSerdes.Long().asInstanceOf[Serde[Long]] + implicit def JavaLong: Serde[java.lang.Long] = JSerdes.Long() + implicit def ByteArray: Serde[Array[Byte]] = JSerdes.ByteArray() + implicit def Bytes: Serde[org.apache.kafka.common.utils.Bytes] = JSerdes.Bytes() + implicit def Float: Serde[Float] = JSerdes.Float().asInstanceOf[Serde[Float]] + implicit def JavaFloat: Serde[java.lang.Float] = JSerdes.Float() + implicit def Double: Serde[Double] = JSerdes.Double().asInstanceOf[Serde[Double]] + implicit def JavaDouble: Serde[java.lang.Double] = JSerdes.Double() + implicit def Integer: Serde[Int] = JSerdes.Integer().asInstanceOf[Serde[Int]] + implicit def JavaInteger: Serde[java.lang.Integer] = JSerdes.Integer() implicit def timeWindowedSerde[T]: WindowedSerdes.TimeWindowedSerde[T] = new WindowedSerdes.TimeWindowedSerde[T]() - implicit def sessionWindowedSerde[T]: WindowedSerdes.SessionWindowedSerde[T] = new WindowedSerdes.SessionWindowedSerde[T]() + implicit def sessionWindowedSerde[T]: WindowedSerdes.SessionWindowedSerde[T] = + new WindowedSerdes.SessionWindowedSerde[T]() def fromFn[T >: Null](serializer: T => Array[Byte], deserializer: Array[Byte] => Option[T]): Serde[T] = JSerdes.serdeFrom( new Serializer[T] { - override def serialize(topic: String, data: T): Array[Byte] = serializer(data) + override def serialize(topic: String, data: T): Array[Byte] = serializer(data) override def configure(configs: util.Map[String, _], isKey: Boolean): Unit = () - override def close(): Unit = () + override def close(): Unit = () }, new Deserializer[T] { - override def deserialize(topic: String, data: Array[Byte]): T = deserializer(data).orNull + override def deserialize(topic: String, data: Array[Byte]): T = deserializer(data).orNull override def configure(configs: util.Map[String, _], isKey: Boolean): Unit = () - override def close(): Unit = () + override def close(): Unit = () } ) def fromFn[T >: Null](serializer: (String, T) => Array[Byte], - deserializer: (String, Array[Byte]) => Option[T]): Serde[T] = + deserializer: (String, Array[Byte]) => Option[T]): Serde[T] = JSerdes.serdeFrom( new Serializer[T] { - override def serialize(topic: String, data: T): Array[Byte] = serializer(topic, data) + override def serialize(topic: String, data: T): Array[Byte] = serializer(topic, data) override def configure(configs: util.Map[String, _], isKey: Boolean): Unit = () - override def close(): Unit = () + override def close(): Unit = () }, new Deserializer[T] { - override def deserialize(topic: String, data: Array[Byte]): T = deserializer(topic, data).orNull + override def deserialize(topic: String, data: Array[Byte]): T = deserializer(topic, data).orNull override def configure(configs: util.Map[String, _], isKey: Boolean): Unit = () - override def close(): Unit = () + override def close(): Unit = () } ) } diff --git a/streams/streams-scala/src/main/scala/org/apache/kafka/streams/scala/StreamsBuilder.scala b/streams/streams-scala/src/main/scala/org/apache/kafka/streams/scala/StreamsBuilder.scala index af342acce3af3..fcec778348007 100644 --- a/streams/streams-scala/src/main/scala/org/apache/kafka/streams/scala/StreamsBuilder.scala +++ b/streams/streams-scala/src/main/scala/org/apache/kafka/streams/scala/StreamsBuilder.scala @@ -31,18 +31,18 @@ import ImplicitConversions._ import scala.collection.JavaConverters._ /** - * Wraps the Java class StreamsBuilder and delegates method calls to the underlying Java object. - */ + * Wraps the Java class StreamsBuilder and delegates method calls to the underlying Java object. + */ class StreamsBuilder(inner: StreamsBuilderJ = new StreamsBuilderJ) { /** * Create a [[kstream.KStream]] from the specified topic. *

                        - * The `implicit Consumed` instance provides the values of `auto.offset.reset` strategy, `TimestampExtractor`, + * The `implicit Consumed` instance provides the values of `auto.offset.reset` strategy, `TimestampExtractor`, * key and value deserializers etc. If the implicit is not found in scope, compiler error will result. *

                        * A convenient alternative is to have the necessary implicit serdes in scope, which will be implicitly - * converted to generate an instance of `Consumed`. @see [[ImplicitConversions]]. + * converted to generate an instance of `Consumed`. @see [[ImplicitConversions]]. * {{{ * // Brings all implicit conversions in scope * import ImplicitConversions._ @@ -88,11 +88,11 @@ class StreamsBuilder(inner: StreamsBuilderJ = new StreamsBuilderJ) { /** * Create a [[kstream.KTable]] from the specified topic. *

                        - * The `implicit Consumed` instance provides the values of `auto.offset.reset` strategy, `TimestampExtractor`, + * The `implicit Consumed` instance provides the values of `auto.offset.reset` strategy, `TimestampExtractor`, * key and value deserializers etc. If the implicit is not found in scope, compiler error will result. *

                        * A convenient alternative is to have the necessary implicit serdes in scope, which will be implicitly - * converted to generate an instance of `Consumed`. @see [[ImplicitConversions]]. + * converted to generate an instance of `Consumed`. @see [[ImplicitConversions]]. * {{{ * // Brings all implicit conversions in scope * import ImplicitConversions._ @@ -123,8 +123,9 @@ class StreamsBuilder(inner: StreamsBuilderJ = new StreamsBuilderJ) { * @see #table(String) * @see `org.apache.kafka.streams.StreamsBuilder#table` */ - def table[K, V](topic: String, materialized: Materialized[K, V, ByteArrayKeyValueStore]) - (implicit consumed: Consumed[K, V]): KTable[K, V] = + def table[K, V](topic: String, materialized: Materialized[K, V, ByteArrayKeyValueStore])( + implicit consumed: Consumed[K, V] + ): KTable[K, V] = inner.table[K, V](topic, consumed, materialized) /** @@ -139,8 +140,8 @@ class StreamsBuilder(inner: StreamsBuilderJ = new StreamsBuilderJ) { inner.globalTable(topic, consumed) /** - * Create a `GlobalKTable` from the specified topic. The resulting `GlobalKTable` will be materialized - * in a local `KeyValueStore` configured with the provided instance of `Materialized`. The serializers + * Create a `GlobalKTable` from the specified topic. The resulting `GlobalKTable` will be materialized + * in a local `KeyValueStore` configured with the provided instance of `Materialized`. The serializers * from the implicit `Consumed` instance will be used. * * @param topic the topic name @@ -148,12 +149,13 @@ class StreamsBuilder(inner: StreamsBuilderJ = new StreamsBuilderJ) { * @return a `GlobalKTable` for the specified topic * @see `org.apache.kafka.streams.StreamsBuilder#globalTable` */ - def globalTable[K, V](topic: String, materialized: Materialized[K, V, ByteArrayKeyValueStore]) - (implicit consumed: Consumed[K, V]): GlobalKTable[K, V] = + def globalTable[K, V](topic: String, materialized: Materialized[K, V, ByteArrayKeyValueStore])( + implicit consumed: Consumed[K, V] + ): GlobalKTable[K, V] = inner.globalTable(topic, consumed, materialized) /** - * Adds a state store to the underlying `Topology`. The store must still be "connected" to a `Processor`, + * Adds a state store to the underlying `Topology`. The store must still be "connected" to a `Processor`, * `Transformer`, or `ValueTransformer` before it can be used. * * @param builder the builder used to obtain this state store `StateStore` instance @@ -164,11 +166,11 @@ class StreamsBuilder(inner: StreamsBuilderJ = new StreamsBuilderJ) { def addStateStore(builder: StoreBuilder[_ <: StateStore]): StreamsBuilderJ = inner.addStateStore(builder) /** - * Adds a global `StateStore` to the topology. Global stores should not be added to `Processor, `Transformer`, + * Adds a global `StateStore` to the topology. Global stores should not be added to `Processor, `Transformer`, * or `ValueTransformer` (in contrast to regular stores). * * @see `org.apache.kafka.streams.StreamsBuilder#addGlobalStore` - */ + */ def addGlobalStore(storeBuilder: StoreBuilder[_ <: StateStore], topic: String, consumed: Consumed[_, _], diff --git a/streams/streams-scala/src/main/scala/org/apache/kafka/streams/scala/kstream/KGroupedStream.scala b/streams/streams-scala/src/main/scala/org/apache/kafka/streams/scala/kstream/KGroupedStream.scala index 0e5abfdd11b8d..f6a22d91812a0 100644 --- a/streams/streams-scala/src/main/scala/org/apache/kafka/streams/scala/kstream/KGroupedStream.scala +++ b/streams/streams-scala/src/main/scala/org/apache/kafka/streams/scala/kstream/KGroupedStream.scala @@ -24,7 +24,6 @@ import org.apache.kafka.streams.kstream.{KGroupedStream => KGroupedStreamJ, _} import org.apache.kafka.streams.scala.ImplicitConversions._ import org.apache.kafka.streams.scala.FunctionConversions._ - /** * Wraps the Java class KGroupedStream and delegates method calls to the underlying Java object. * @@ -41,7 +40,7 @@ class KGroupedStream[K, V](val inner: KGroupedStreamJ[K, V]) { * The result is written into a local `KeyValueStore` (which is basically an ever-updating materialized view) * provided by the given `materialized`. * - * @param materialized an instance of `Materialized` used to materialize a state store. + * @param materialized an instance of `Materialized` used to materialize a state store. * @return a [[KTable]] that contains "update" records with unmodified keys and `Long` values that * represent the latest (rolling) count (i.e., number of records) for each key * @see `org.apache.kafka.streams.kstream.KGroupedStream#count` @@ -55,8 +54,8 @@ class KGroupedStream[K, V](val inner: KGroupedStreamJ[K, V]) { /** * Combine the values of records in this stream by the grouped key. * - * @param reducer a function `(V, V) => V` that computes a new aggregate result. - * @param materialized an instance of `Materialized` used to materialize a state store. + * @param reducer a function `(V, V) => V` that computes a new aggregate result. + * @param materialized an instance of `Materialized` used to materialize a state store. * @return a [[KTable]] that contains "update" records with unmodified keys, and values that represent the * latest (rolling) aggregate for each key * @see `org.apache.kafka.streams.kstream.KGroupedStream#reduce` diff --git a/streams/streams-scala/src/main/scala/org/apache/kafka/streams/scala/kstream/KGroupedTable.scala b/streams/streams-scala/src/main/scala/org/apache/kafka/streams/scala/kstream/KGroupedTable.scala index 99bc83e1ba0e5..76ea9ed75558a 100644 --- a/streams/streams-scala/src/main/scala/org/apache/kafka/streams/scala/kstream/KGroupedTable.scala +++ b/streams/streams-scala/src/main/scala/org/apache/kafka/streams/scala/kstream/KGroupedTable.scala @@ -39,7 +39,7 @@ class KGroupedTable[K, V](inner: KGroupedTableJ[K, V]) { * Count number of records of the original [[KTable]] that got [[KTable#groupBy]] to * the same key into a new instance of [[KTable]]. * - * @param materialized an instance of `Materialized` used to materialize a state store. + * @param materialized an instance of `Materialized` used to materialize a state store. * @return a [[KTable]] that contains "update" records with unmodified keys and `Long` values that * represent the latest (rolling) count (i.e., number of records) for each key * @see `org.apache.kafka.streams.kstream.KGroupedTable#count` diff --git a/streams/streams-scala/src/main/scala/org/apache/kafka/streams/scala/kstream/KStream.scala b/streams/streams-scala/src/main/scala/org/apache/kafka/streams/scala/kstream/KStream.scala index 0f1fc82a48984..8e4c9aa7eca9a 100644 --- a/streams/streams-scala/src/main/scala/org/apache/kafka/streams/scala/kstream/KStream.scala +++ b/streams/streams-scala/src/main/scala/org/apache/kafka/streams/scala/kstream/KStream.scala @@ -22,7 +22,7 @@ package kstream import org.apache.kafka.streams.KeyValue import org.apache.kafka.streams.kstream.{KStream => KStreamJ, _} -import org.apache.kafka.streams.processor.{Processor, ProcessorContext, ProcessorSupplier, TopicNameExtractor} +import org.apache.kafka.streams.processor.{Processor, ProcessorSupplier, TopicNameExtractor} import org.apache.kafka.streams.scala.ImplicitConversions._ import org.apache.kafka.streams.scala.FunctionConversions._ @@ -31,8 +31,8 @@ import scala.collection.JavaConverters._ /** * Wraps the Java class [[org.apache.kafka.streams.kstream.KStream]] and delegates method calls to the underlying Java object. * - * @param [K] Type of keys - * @param [V] Type of values + * @tparam K Type of keys + * @tparam V Type of values * @param inner The underlying Java abstraction for KStream * * @see `org.apache.kafka.streams.kstream.KStream` @@ -46,7 +46,7 @@ class KStream[K, V](val inner: KStreamJ[K, V]) { * @param predicate a filter that is applied to each record * @return a [[KStream]] that contains only those records that satisfy the given predicate * @see `org.apache.kafka.streams.kstream.KStream#filter` - */ + */ def filter(predicate: (K, V) => Boolean): KStream[K, V] = inner.filter(predicate.asPredicate) @@ -57,7 +57,7 @@ class KStream[K, V](val inner: KStreamJ[K, V]) { * @param predicate a filter that is applied to each record * @return a [[KStream]] that contains only those records that do not satisfy the given predicate * @see `org.apache.kafka.streams.kstream.KStream#filterNot` - */ + */ def filterNot(predicate: (K, V) => Boolean): KStream[K, V] = inner.filterNot(predicate.asPredicate) @@ -70,7 +70,7 @@ class KStream[K, V](val inner: KStreamJ[K, V]) { * @param mapper a function `(K, V) => KR` that computes a new key for each record * @return a [[KStream]] that contains records with new key (possibly of different type) and unmodified value * @see `org.apache.kafka.streams.kstream.KStream#selectKey` - */ + */ def selectKey[KR](mapper: (K, V) => KR): KStream[KR, V] = inner.selectKey[KR](mapper.asKeyValueMapper) @@ -83,11 +83,9 @@ class KStream[K, V](val inner: KStreamJ[K, V]) { * @param mapper a function `(K, V) => (KR, VR)` that computes a new output record * @return a [[KStream]] that contains records with new key and value (possibly both of different type) * @see `org.apache.kafka.streams.kstream.KStream#map` - */ - def map[KR, VR](mapper: (K, V) => (KR, VR)): KStream[KR, VR] = { - val kvMapper = mapper.tupled andThen tuple2ToKeyValue - inner.map[KR, VR](((k: K, v: V) => kvMapper(k, v)).asKeyValueMapper) - } + */ + def map[KR, VR](mapper: (K, V) => (KR, VR)): KStream[KR, VR] = + inner.map[KR, VR](mapper.asKeyValueMapper) /** * Transform the value of each input record into a new value (with possible new type) of the output record. @@ -97,7 +95,7 @@ class KStream[K, V](val inner: KStreamJ[K, V]) { * @param mapper, a function `V => VR` that computes a new output value * @return a [[KStream]] that contains records with unmodified key and new values (possibly of different type) * @see `org.apache.kafka.streams.kstream.KStream#mapValues` - */ + */ def mapValues[VR](mapper: V => VR): KStream[K, VR] = inner.mapValues[VR](mapper.asValueMapper) @@ -109,7 +107,7 @@ class KStream[K, V](val inner: KStreamJ[K, V]) { * @param mapper, a function `(K, V) => VR` that computes a new output value * @return a [[KStream]] that contains records with unmodified key and new values (possibly of different type) * @see `org.apache.kafka.streams.kstream.KStream#mapValues` - */ + */ def mapValues[VR](mapper: (K, V) => VR): KStream[K, VR] = inner.mapValues[VR](mapper.asValueMapperWithKey) @@ -122,10 +120,10 @@ class KStream[K, V](val inner: KStreamJ[K, V]) { * @param mapper function `(K, V) => Iterable[(KR, VR)]` that computes the new output records * @return a [[KStream]] that contains more or less records with new key and value (possibly of different type) * @see `org.apache.kafka.streams.kstream.KStream#flatMap` - */ + */ def flatMap[KR, VR](mapper: (K, V) => Iterable[(KR, VR)]): KStream[KR, VR] = { - val kvMapper = mapper.tupled andThen (iter => iter.map(tuple2ToKeyValue).asJava) - inner.flatMap[KR, VR](((k: K, v: V) => kvMapper(k , v)).asKeyValueMapper) + val kvMapper = mapper.tupled.andThen(_.map(tuple2ToKeyValue).asJava) + inner.flatMap[KR, VR](((k: K, v: V) => kvMapper(k, v)).asKeyValueMapper) } /** @@ -139,7 +137,7 @@ class KStream[K, V](val inner: KStreamJ[K, V]) { * @param mapper a function `V => Iterable[VR]` that computes the new output values * @return a [[KStream]] that contains more or less records with unmodified keys and new values of different type * @see `org.apache.kafka.streams.kstream.KStream#flatMapValues` - */ + */ def flatMapValues[VR](mapper: V => Iterable[VR]): KStream[K, VR] = inner.flatMapValues[VR](mapper.asValueMapper) @@ -154,7 +152,7 @@ class KStream[K, V](val inner: KStreamJ[K, V]) { * @param mapper a function `(K, V) => Iterable[VR]` that computes the new output values * @return a [[KStream]] that contains more or less records with unmodified keys and new values of different type * @see `org.apache.kafka.streams.kstream.KStream#flatMapValues` - */ + */ def flatMapValues[VR](mapper: (K, V) => Iterable[VR]): KStream[K, VR] = inner.flatMapValues[VR](mapper.asValueMapperWithKey) @@ -167,27 +165,28 @@ class KStream[K, V](val inner: KStreamJ[K, V]) { def print(printed: Printed[K, V]): Unit = inner.print(printed) /** - * Perform an action on each record of 'KStream` + * Perform an action on each record of `KStream` * * @param action an action to perform on each record * @see `org.apache.kafka.streams.kstream.KStream#foreach` */ def foreach(action: (K, V) => Unit): Unit = - inner.foreach((k: K, v: V) => action(k, v)) + inner.foreach(action.asForeachAction) /** - * Creates an array of {@code KStream} from this stream by branching the records in the original stream based on + * Creates an array of `KStream` from this stream by branching the records in the original stream based on * the supplied predicates. * * @param predicates the ordered list of functions that return a Boolean * @return multiple distinct substreams of this [[KStream]] * @see `org.apache.kafka.streams.kstream.KStream#branch` */ - def branch(predicates: (K, V) => Boolean*): Array[KStream[K, V]] = + //noinspection ScalaUnnecessaryParentheses + def branch(predicates: ((K, V) => Boolean)*): Array[KStream[K, V]] = inner.branch(predicates.map(_.asPredicate): _*).map(kstream => wrapKStream(kstream)) /** - * Materialize this stream to a topic and creates a new [[KStream]] from the topic using the `Produced` instance for + * Materialize this stream to a topic and creates a new [[KStream]] from the topic using the `Produced` instance for * configuration of the `Serde key serde`, `Serde value serde`, and `StreamPartitioner` *

                        * The user can either supply the `Produced` instance as an implicit in scope or she can also provide implicit @@ -211,7 +210,7 @@ class KStream[K, V](val inner: KStreamJ[K, V]) { * }}} * * @param topic the topic name - * @param (implicit) produced the instance of Produced that gives the serdes and `StreamPartitioner` + * @param produced the instance of Produced that gives the serdes and `StreamPartitioner` * @return a [[KStream]] that contains the exact same (and potentially repartitioned) records as this [[KStream]] * @see `org.apache.kafka.streams.kstream.KStream#through` */ @@ -219,7 +218,7 @@ class KStream[K, V](val inner: KStreamJ[K, V]) { inner.through(topic, produced) /** - * Materialize this stream to a topic using the `Produced` instance for + * Materialize this stream to a topic using the `Produced` instance for * configuration of the `Serde key serde`, `Serde value serde`, and `StreamPartitioner` *

                        * The user can either supply the `Produced` instance as an implicit in scope or she can also provide implicit @@ -243,89 +242,74 @@ class KStream[K, V](val inner: KStreamJ[K, V]) { * }}} * * @param topic the topic name - * @param (implicit) produced the instance of Produced that gives the serdes and `StreamPartitioner` + * @param produced the instance of Produced that gives the serdes and `StreamPartitioner` * @see `org.apache.kafka.streams.kstream.KStream#to` */ def to(topic: String)(implicit produced: Produced[K, V]): Unit = inner.to(topic, produced) /** - * Dynamically materialize this stream to topics using the `Produced` instance for - * configuration of the `Serde key serde`, `Serde value serde`, and `StreamPartitioner`. - * The topic names for each record to send to is dynamically determined based on the given mapper. - *

                        - * The user can either supply the `Produced` instance as an implicit in scope or she can also provide implicit - * key and value serdes that will be converted to a `Produced` instance implicitly. - *

                        - * {{{ - * Example: - * - * // brings implicit serdes in scope - * import Serdes._ - * - * //.. - * val clicksPerRegion: KTable[String, Long] = //.. - * - * // Implicit serdes in scope will generate an implicit Produced instance, which - * // will be passed automatically to the call of through below - * clicksPerRegion.to(topicChooser) - * - * // Similarly you can create an implicit Produced and it will be passed implicitly - * // to the through call - * }}} - * - * @param extractor the extractor to determine the name of the Kafka topic to write to for reach record - * @param (implicit) produced the instance of Produced that gives the serdes and `StreamPartitioner` - * @see `org.apache.kafka.streams.kstream.KStream#to` - */ + * Dynamically materialize this stream to topics using the `Produced` instance for + * configuration of the `Serde key serde`, `Serde value serde`, and `StreamPartitioner`. + * The topic names for each record to send to is dynamically determined based on the given mapper. + *

                        + * The user can either supply the `Produced` instance as an implicit in scope or she can also provide implicit + * key and value serdes that will be converted to a `Produced` instance implicitly. + *

                        + * {{{ + * Example: + * + * // brings implicit serdes in scope + * import Serdes._ + * + * //.. + * val clicksPerRegion: KTable[String, Long] = //.. + * + * // Implicit serdes in scope will generate an implicit Produced instance, which + * // will be passed automatically to the call of through below + * clicksPerRegion.to(topicChooser) + * + * // Similarly you can create an implicit Produced and it will be passed implicitly + * // to the through call + * }}} + * + * @param extractor the extractor to determine the name of the Kafka topic to write to for reach record + * @param produced the instance of Produced that gives the serdes and `StreamPartitioner` + * @see `org.apache.kafka.streams.kstream.KStream#to` + */ def to(extractor: TopicNameExtractor[K, V])(implicit produced: Produced[K, V]): Unit = inner.to(extractor, produced) /** * Transform each record of the input stream into zero or more records in the output stream (both key and value type * can be altered arbitrarily). - * A `Transformer` is applied to each input record and computes zero or more output records. In order to assign a - * state, the state must be created and registered beforehand via stores added via `addStateStore` or `addGlobalStore` + * A `Transformer` (provided by the given `TransformerSupplier`) is applied to each input record + * and computes zero or more output records. + * In order to assign a state, the state must be created and registered + * beforehand via stores added via `addStateStore` or `addGlobalStore` * before they can be connected to the `Transformer` * - * @param transformer the `Transformer` instance + * @param transformerSupplier the `TransformerSuplier` that generates `Transformer` * @param stateStoreNames the names of the state stores used by the processor * @return a [[KStream]] that contains more or less records with new key and value (possibly of different type) * @see `org.apache.kafka.streams.kstream.KStream#transform` - */ - def transform[K1, V1](transformer: Transformer[K, V, (K1, V1)], - stateStoreNames: String*): KStream[K1, V1] = { - val transformerSupplierJ: TransformerSupplier[K, V, KeyValue[K1, V1]] = new TransformerSupplier[K, V, KeyValue[K1, V1]] { - override def get(): Transformer[K, V, KeyValue[K1, V1]] = { - new Transformer[K, V, KeyValue[K1, V1]] { - override def transform(key: K, value: V): KeyValue[K1, V1] = { - transformer.transform(key, value) match { - case (k1, v1) => KeyValue.pair(k1, v1) - case _ => null - } - } - - override def init(context: ProcessorContext): Unit = transformer.init(context) - - override def close(): Unit = transformer.close() - } - } - } - inner.transform(transformerSupplierJ, stateStoreNames: _*) - } + */ + def transform[K1, V1](transformerSupplier: TransformerSupplier[K, V, KeyValue[K1, V1]], + stateStoreNames: String*): KStream[K1, V1] = + inner.transform(transformerSupplier, stateStoreNames: _*) /** * Transform the value of each input record into a new value (with possible new type) of the output record. * A `ValueTransformer` (provided by the given `ValueTransformerSupplier`) is applied to each input * record value and computes a new value for it. - * In order to assign a state, the state must be created and registered - * beforehand via stores added via `addStateStore` or `addGlobalStore` before they can be connected to the `Transformer` + * In order to assign a state, the state must be created and registered + * beforehand via stores added via `addStateStore` or `addGlobalStore` before they can be connected to the `Transformer` * * @param valueTransformerSupplier a instance of `ValueTransformerSupplier` that generates a `ValueTransformer` * @param stateStoreNames the names of the state stores used by the processor * @return a [[KStream]] that contains records with unmodified key and new values (possibly of different type) * @see `org.apache.kafka.streams.kstream.KStream#transformValues` - */ + */ def transformValues[VR](valueTransformerSupplier: ValueTransformerSupplier[V, VR], stateStoreNames: String*): KStream[K, VR] = inner.transformValues[VR](valueTransformerSupplier, stateStoreNames: _*) @@ -334,30 +318,30 @@ class KStream[K, V](val inner: KStreamJ[K, V]) { * Transform the value of each input record into a new value (with possible new type) of the output record. * A `ValueTransformer` (provided by the given `ValueTransformerSupplier`) is applied to each input * record value and computes a new value for it. - * In order to assign a state, the state must be created and registered - * beforehand via stores added via `addStateStore` or `addGlobalStore` before they can be connected to the `Transformer` + * In order to assign a state, the state must be created and registered + * beforehand via stores added via `addStateStore` or `addGlobalStore` before they can be connected to the `Transformer` * * @param valueTransformerSupplier a instance of `ValueTransformerWithKeySupplier` that generates a `ValueTransformerWithKey` * @param stateStoreNames the names of the state stores used by the processor * @return a [[KStream]] that contains records with unmodified key and new values (possibly of different type) * @see `org.apache.kafka.streams.kstream.KStream#transformValues` - */ + */ def transformValues[VR](valueTransformerSupplier: ValueTransformerWithKeySupplier[K, V, VR], - stateStoreNames: String*): KStream[K, VR] = { + stateStoreNames: String*): KStream[K, VR] = inner.transformValues[VR](valueTransformerSupplier, stateStoreNames: _*) - } /** * Process all records in this stream, one record at a time, by applying a `Processor` (provided by the given * `processorSupplier`). - * In order to assign a state, the state must be created and registered - * beforehand via stores added via `addStateStore` or `addGlobalStore` before they can be connected to the `Transformer` + * In order to assign a state, the state must be created and registered + * beforehand via stores added via `addStateStore` or `addGlobalStore` before they can be connected to the `Transformer` * - * @param processorSupplier a function that generates a [[org.apache.kafka.stream.Processor]] + * @param processorSupplier a function that generates a [[org.apache.kafka.streams.processor.Processor]] * @param stateStoreNames the names of the state store used by the processor * @see `org.apache.kafka.streams.kstream.KStream#process` - */ + */ def process(processorSupplier: () => Processor[K, V], stateStoreNames: String*): Unit = { + //noinspection ConvertExpressionToSAM // because of the 2.11 build val processorSupplierJ: ProcessorSupplier[K, V] = new ProcessorSupplier[K, V] { override def get(): Processor[K, V] = processorSupplier() } @@ -365,7 +349,7 @@ class KStream[K, V](val inner: KStreamJ[K, V]) { } /** - * Group the records by their current key into a [[KGroupedStream]] + * Group the records by their current key into a [[KGroupedStream]] *

                        * The user can either supply the `Serialized` instance as an implicit in scope or she can also provide an implicit * serdes that will be converted to a `Serialized` instance implicitly. @@ -390,10 +374,10 @@ class KStream[K, V](val inner: KStreamJ[K, V]) { * // to the groupByKey call * }}} * - * @param (implicit) serialized the instance of Serialized that gives the serdes + * @param serialized the instance of Serialized that gives the serdes * @return a [[KGroupedStream]] that contains the grouped records of the original [[KStream]] * @see `org.apache.kafka.streams.kstream.KStream#groupByKey` - */ + */ def groupByKey(implicit serialized: Serialized[K, V]): KGroupedStream[K, V] = inner.groupByKey(serialized) @@ -427,18 +411,18 @@ class KStream[K, V](val inner: KStreamJ[K, V]) { * @param selector a function that computes a new key for grouping * @return a [[KGroupedStream]] that contains the grouped records of the original [[KStream]] * @see `org.apache.kafka.streams.kstream.KStream#groupBy` - */ + */ def groupBy[KR](selector: (K, V) => KR)(implicit serialized: Serialized[KR, V]): KGroupedStream[KR, V] = inner.groupBy(selector.asKeyValueMapper, serialized) /** - * Join records of this stream with another [[KStream]]'s records using windowed inner equi join with + * Join records of this stream with another [[KStream]]'s records using windowed inner equi join with * serializers and deserializers supplied by the implicit `Joined` instance. * * @param otherStream the [[KStream]] to be joined with this stream * @param joiner a function that computes the join result for a pair of matching records * @param windows the specification of the `JoinWindows` - * @param joined an implicit `Joined` instance that defines the serdes to be used to serialize/deserialize + * @param joined an implicit `Joined` instance that defines the serdes to be used to serialize/deserialize * inputs and outputs of the joined streams. Instead of `Joined`, the user can also supply * key serde, value serde and other value serde in implicit scope and they will be * converted to the instance of `Joined` through implicit conversion @@ -453,17 +437,17 @@ class KStream[K, V](val inner: KStreamJ[K, V]) { inner.join[VO, VR](otherStream.inner, joiner.asValueJoiner, windows, joined) /** - * Join records of this stream with another [[KTable]]'s records using inner equi join with + * Join records of this stream with another [[KTable]]'s records using inner equi join with * serializers and deserializers supplied by the implicit `Joined` instance. * * @param table the [[KTable]] to be joined with this stream * @param joiner a function that computes the join result for a pair of matching records - * @param joined an implicit `Joined` instance that defines the serdes to be used to serialize/deserialize + * @param joined an implicit `Joined` instance that defines the serdes to be used to serialize/deserialize * inputs and outputs of the joined streams. Instead of `Joined`, the user can also supply * key serde, value serde and other value serde in implicit scope and they will be * converted to the instance of `Joined` through implicit conversion * @return a [[KStream]] that contains join-records for each key and values computed by the given `joiner`, - * one for each matched record-pair with the same key + * one for each matched record-pair with the same key * @see `org.apache.kafka.streams.kstream.KStream#join` */ def join[VT, VR](table: KTable[K, VT])(joiner: (V, VT) => VR)(implicit joined: Joined[K, V, VT]): KStream[K, VR] = @@ -479,7 +463,7 @@ class KStream[K, V](val inner: KStreamJ[K, V]) { * @return a [[KStream]] that contains join-records for each key and values computed by the given `joiner`, * one output for each input [[KStream]] record * @see `org.apache.kafka.streams.kstream.KStream#join` - */ + */ def join[GK, GV, RV](globalKTable: GlobalKTable[GK, GV])( keyValueMapper: (K, V) => GK, joiner: (V, GV) => RV @@ -491,20 +475,20 @@ class KStream[K, V](val inner: KStreamJ[K, V]) { ) /** - * Join records of this stream with another [[KStream]]'s records using windowed left equi join with + * Join records of this stream with another [[KStream]]'s records using windowed left equi join with * serializers and deserializers supplied by the implicit `Joined` instance. * * @param otherStream the [[KStream]] to be joined with this stream * @param joiner a function that computes the join result for a pair of matching records * @param windows the specification of the `JoinWindows` - * @param joined an implicit `Joined` instance that defines the serdes to be used to serialize/deserialize + * @param joined an implicit `Joined` instance that defines the serdes to be used to serialize/deserialize * inputs and outputs of the joined streams. Instead of `Joined`, the user can also supply * key serde, value serde and other value serde in implicit scope and they will be * converted to the instance of `Joined` through implicit conversion * @return a [[KStream]] that contains join-records for each key and values computed by the given `joiner`, * one for each matched record-pair with the same key and within the joining window intervals * @see `org.apache.kafka.streams.kstream.KStream#leftJoin` - */ + */ def leftJoin[VO, VR](otherStream: KStream[K, VO])( joiner: (V, VO) => VR, windows: JoinWindows @@ -512,19 +496,19 @@ class KStream[K, V](val inner: KStreamJ[K, V]) { inner.leftJoin[VO, VR](otherStream.inner, joiner.asValueJoiner, windows, joined) /** - * Join records of this stream with another [[KTable]]'s records using left equi join with + * Join records of this stream with another [[KTable]]'s records using left equi join with * serializers and deserializers supplied by the implicit `Joined` instance. * * @param table the [[KTable]] to be joined with this stream * @param joiner a function that computes the join result for a pair of matching records - * @param joined an implicit `Joined` instance that defines the serdes to be used to serialize/deserialize + * @param joined an implicit `Joined` instance that defines the serdes to be used to serialize/deserialize * inputs and outputs of the joined streams. Instead of `Joined`, the user can also supply * key serde, value serde and other value serde in implicit scope and they will be * converted to the instance of `Joined` through implicit conversion * @return a [[KStream]] that contains join-records for each key and values computed by the given `joiner`, - * one for each matched record-pair with the same key + * one for each matched record-pair with the same key * @see `org.apache.kafka.streams.kstream.KStream#leftJoin` - */ + */ def leftJoin[VT, VR](table: KTable[K, VT])(joiner: (V, VT) => VR)(implicit joined: Joined[K, V, VT]): KStream[K, VR] = inner.leftJoin[VT, VR](table.inner, joiner.asValueJoiner, joined) @@ -538,7 +522,7 @@ class KStream[K, V](val inner: KStreamJ[K, V]) { * @return a [[KStream]] that contains join-records for each key and values computed by the given `joiner`, * one output for each input [[KStream]] record * @see `org.apache.kafka.streams.kstream.KStream#leftJoin` - */ + */ def leftJoin[GK, GV, RV](globalKTable: GlobalKTable[GK, GV])( keyValueMapper: (K, V) => GK, joiner: (V, GV) => RV @@ -546,20 +530,20 @@ class KStream[K, V](val inner: KStreamJ[K, V]) { inner.leftJoin[GK, GV, RV](globalKTable, keyValueMapper.asKeyValueMapper, joiner.asValueJoiner) /** - * Join records of this stream with another [[KStream]]'s records using windowed outer equi join with + * Join records of this stream with another [[KStream]]'s records using windowed outer equi join with * serializers and deserializers supplied by the implicit `Joined` instance. * * @param otherStream the [[KStream]] to be joined with this stream * @param joiner a function that computes the join result for a pair of matching records * @param windows the specification of the `JoinWindows` - * @param joined an implicit `Joined` instance that defines the serdes to be used to serialize/deserialize + * @param joined an implicit `Joined` instance that defines the serdes to be used to serialize/deserialize * inputs and outputs of the joined streams. Instead of `Joined`, the user can also supply * key serde, value serde and other value serde in implicit scope and they will be * converted to the instance of `Joined` through implicit conversion * @return a [[KStream]] that contains join-records for each key and values computed by the given `joiner`, * one for each matched record-pair with the same key and within the joining window intervals * @see `org.apache.kafka.streams.kstream.KStream#outerJoin` - */ + */ def outerJoin[VO, VR](otherStream: KStream[K, VO])( joiner: (V, VO) => VR, windows: JoinWindows @@ -569,8 +553,8 @@ class KStream[K, V](val inner: KStreamJ[K, V]) { /** * Merge this stream and the given stream into one larger stream. *

                        - * There is no ordering guarantee between records from this `KStream` and records from the provided `KStream` - * in the merged stream. Relative order is preserved within each input stream though (ie, records within + * There is no ordering guarantee between records from this `KStream` and records from the provided `KStream` + * in the merged stream. Relative order is preserved within each input stream though (ie, records within * one input stream are processed in order). * * @param stream a stream which is to be merged into this stream @@ -580,7 +564,7 @@ class KStream[K, V](val inner: KStreamJ[K, V]) { def merge(stream: KStream[K, V]): KStream[K, V] = inner.merge(stream.inner) /** - * Perform an action on each record of {@code KStream}. + * Perform an action on each record of `KStream`. *

                        * Peek is a non-terminal operation that triggers a side effect (such as logging or statistics collection) * and returns an unchanged stream. @@ -589,5 +573,5 @@ class KStream[K, V](val inner: KStreamJ[K, V]) { * @see `org.apache.kafka.streams.kstream.KStream#peek` */ def peek(action: (K, V) => Unit): KStream[K, V] = - inner.peek((k: K, v: V) => action(k, v)) + inner.peek(action.asForeachAction) } diff --git a/streams/streams-scala/src/main/scala/org/apache/kafka/streams/scala/kstream/KTable.scala b/streams/streams-scala/src/main/scala/org/apache/kafka/streams/scala/kstream/KTable.scala index cff1844121dd3..d41496fb21cd2 100644 --- a/streams/streams-scala/src/main/scala/org/apache/kafka/streams/scala/kstream/KTable.scala +++ b/streams/streams-scala/src/main/scala/org/apache/kafka/streams/scala/kstream/KTable.scala @@ -20,6 +20,7 @@ package org.apache.kafka.streams.scala package kstream +import org.apache.kafka.common.serialization.Serde import org.apache.kafka.common.utils.Bytes import org.apache.kafka.streams.kstream.{KTable => KTableJ, _} import org.apache.kafka.streams.scala.ImplicitConversions._ @@ -44,10 +45,9 @@ class KTable[K, V](val inner: KTableJ[K, V]) { * @param predicate a filter that is applied to each record * @return a [[KTable]] that contains only those records that satisfy the given predicate * @see `org.apache.kafka.streams.kstream.KTable#filter` - */ - def filter(predicate: (K, V) => Boolean): KTable[K, V] = { - inner.filter(predicate(_, _)) - } + */ + def filter(predicate: (K, V) => Boolean): KTable[K, V] = + inner.filter(predicate.asPredicate) /** * Create a new [[KTable]] that consists all records of this [[KTable]] which satisfies the given @@ -55,12 +55,11 @@ class KTable[K, V](val inner: KTableJ[K, V]) { * * @param predicate a filter that is applied to each record * @param materialized a `Materialized` that describes how the `StateStore` for the resulting [[KTable]] - * should be materialized. + * should be materialized. * @return a [[KTable]] that contains only those records that satisfy the given predicate * @see `org.apache.kafka.streams.kstream.KTable#filter` - */ - def filter(predicate: (K, V) => Boolean, - materialized: Materialized[K, V, ByteArrayKeyValueStore]): KTable[K, V] = + */ + def filter(predicate: (K, V) => Boolean, materialized: Materialized[K, V, ByteArrayKeyValueStore]): KTable[K, V] = inner.filter(predicate.asPredicate, materialized) /** @@ -70,9 +69,9 @@ class KTable[K, V](val inner: KTableJ[K, V]) { * @param predicate a filter that is applied to each record * @return a [[KTable]] that contains only those records that do not satisfy the given predicate * @see `org.apache.kafka.streams.kstream.KTable#filterNot` - */ + */ def filterNot(predicate: (K, V) => Boolean): KTable[K, V] = - inner.filterNot(predicate(_, _)) + inner.filterNot(predicate.asPredicate) /** * Create a new [[KTable]] that consists all records of this [[KTable]] which do not satisfy the given @@ -80,12 +79,11 @@ class KTable[K, V](val inner: KTableJ[K, V]) { * * @param predicate a filter that is applied to each record * @param materialized a `Materialized` that describes how the `StateStore` for the resulting [[KTable]] - * should be materialized. + * should be materialized. * @return a [[KTable]] that contains only those records that do not satisfy the given predicate * @see `org.apache.kafka.streams.kstream.KTable#filterNot` - */ - def filterNot(predicate: (K, V) => Boolean, - materialized: Materialized[K, V, ByteArrayKeyValueStore]): KTable[K, V] = + */ + def filterNot(predicate: (K, V) => Boolean, materialized: Materialized[K, V, ByteArrayKeyValueStore]): KTable[K, V] = inner.filterNot(predicate.asPredicate, materialized) /** @@ -97,7 +95,7 @@ class KTable[K, V](val inner: KTableJ[K, V]) { * @param mapper, a function `V => VR` that computes a new output value * @return a [[KTable]] that contains records with unmodified key and new values (possibly of different type) * @see `org.apache.kafka.streams.kstream.KTable#mapValues` - */ + */ def mapValues[VR](mapper: V => VR): KTable[K, VR] = inner.mapValues[VR](mapper.asValueMapper) @@ -109,12 +107,11 @@ class KTable[K, V](val inner: KTableJ[K, V]) { * * @param mapper, a function `V => VR` that computes a new output value * @param materialized a `Materialized` that describes how the `StateStore` for the resulting [[KTable]] - * should be materialized. + * should be materialized. * @return a [[KTable]] that contains records with unmodified key and new values (possibly of different type) * @see `org.apache.kafka.streams.kstream.KTable#mapValues` - */ - def mapValues[VR](mapper: V => VR, - materialized: Materialized[K, VR, ByteArrayKeyValueStore]): KTable[K, VR] = + */ + def mapValues[VR](mapper: V => VR, materialized: Materialized[K, VR, ByteArrayKeyValueStore]): KTable[K, VR] = inner.mapValues[VR](mapper.asValueMapper, materialized) /** @@ -126,7 +123,7 @@ class KTable[K, V](val inner: KTableJ[K, V]) { * @param mapper, a function `(K, V) => VR` that computes a new output value * @return a [[KTable]] that contains records with unmodified key and new values (possibly of different type) * @see `org.apache.kafka.streams.kstream.KTable#mapValues` - */ + */ def mapValues[VR](mapper: (K, V) => VR): KTable[K, VR] = inner.mapValues[VR](mapper.asValueMapperWithKey) @@ -138,12 +135,11 @@ class KTable[K, V](val inner: KTableJ[K, V]) { * * @param mapper, a function `(K, V) => VR` that computes a new output value * @param materialized a `Materialized` that describes how the `StateStore` for the resulting [[KTable]] - * should be materialized. + * should be materialized. * @return a [[KTable]] that contains records with unmodified key and new values (possibly of different type) * @see `org.apache.kafka.streams.kstream.KTable#mapValues` - */ - def mapValues[VR](mapper: (K, V) => VR, - materialized: Materialized[K, VR, ByteArrayKeyValueStore]): KTable[K, VR] = + */ + def mapValues[VR](mapper: (K, V) => VR, materialized: Materialized[K, VR, ByteArrayKeyValueStore]): KTable[K, VR] = inner.mapValues[VR](mapper.asValueMapperWithKey) /** @@ -165,57 +161,55 @@ class KTable[K, V](val inner: KTableJ[K, V]) { inner.toStream[KR](mapper.asKeyValueMapper) /** - * Create a new `KTable` by transforming the value of each record in this `KTable` into a new value, (with possibly new type). - * Transform the value of each input record into a new value (with possible new type) of the output record. - * A `ValueTransformerWithKey` (provided by the given `ValueTransformerWithKeySupplier`) is applied to each input - * record value and computes a new value for it. - * This is similar to `#mapValues(ValueMapperWithKey)`, but more flexible, allowing access to additional state-stores, - * and to the `ProcessorContext`. - * If the downstream topology uses aggregation functions, (e.g. `KGroupedTable#reduce`, `KGroupedTable#aggregate`, etc), - * care must be taken when dealing with state, (either held in state-stores or transformer instances), to ensure correct - * aggregate results. - * In contrast, if the resulting KTable is materialized, (cf. `#transformValues(ValueTransformerWithKeySupplier, Materialized, String...)`), - * such concerns are handled for you. - * In order to assign a state, the state must be created and registered - * beforehand via stores added via `addStateStore` or `addGlobalStore` before they can be connected to the `Transformer` - * - * @param valueTransformerWithKeySupplier a instance of `ValueTransformerWithKeySupplier` that generates a `ValueTransformerWithKey`. - * At least one transformer instance will be created per streaming task. - * Transformer implementations doe not need to be thread-safe. - * @param stateStoreNames the names of the state stores used by the processor - * @return a [[KStream]] that contains records with unmodified key and new values (possibly of different type) - * @see `org.apache.kafka.streams.kstream.KStream#transformValues` - */ + * Create a new `KTable` by transforming the value of each record in this `KTable` into a new value, (with possibly new type). + * Transform the value of each input record into a new value (with possible new type) of the output record. + * A `ValueTransformerWithKey` (provided by the given `ValueTransformerWithKeySupplier`) is applied to each input + * record value and computes a new value for it. + * This is similar to `#mapValues(ValueMapperWithKey)`, but more flexible, allowing access to additional state-stores, + * and to the `ProcessorContext`. + * If the downstream topology uses aggregation functions, (e.g. `KGroupedTable#reduce`, `KGroupedTable#aggregate`, etc), + * care must be taken when dealing with state, (either held in state-stores or transformer instances), to ensure correct + * aggregate results. + * In contrast, if the resulting KTable is materialized, (cf. `#transformValues(ValueTransformerWithKeySupplier, Materialized, String...)`), + * such concerns are handled for you. + * In order to assign a state, the state must be created and registered + * beforehand via stores added via `addStateStore` or `addGlobalStore` before they can be connected to the `Transformer` + * + * @param valueTransformerWithKeySupplier a instance of `ValueTransformerWithKeySupplier` that generates a `ValueTransformerWithKey`. + * At least one transformer instance will be created per streaming task. + * Transformer implementations doe not need to be thread-safe. + * @param stateStoreNames the names of the state stores used by the processor + * @return a [[KStream]] that contains records with unmodified key and new values (possibly of different type) + * @see `org.apache.kafka.streams.kstream.KStream#transformValues` + */ def transformValues[VR](valueTransformerWithKeySupplier: ValueTransformerWithKeySupplier[K, V, VR], - stateStoreNames: String*): KTable[K, VR] = { + stateStoreNames: String*): KTable[K, VR] = inner.transformValues[VR](valueTransformerWithKeySupplier, stateStoreNames: _*) - } /** - * Create a new `KTable` by transforming the value of each record in this `KTable` into a new value, (with possibly new type). - * A `ValueTransformer` (provided by the given `ValueTransformerSupplier`) is applied to each input - * record value and computes a new value for it. - * This is similar to `#mapValues(ValueMapperWithKey)`, but more flexible, allowing stateful, rather than stateless, - * record-by-record operation, access to additional state-stores, and access to the `ProcessorContext`. - * In order to assign a state, the state must be created and registered - * beforehand via stores added via `addStateStore` or `addGlobalStore` before they can be connected to the `Transformer` - * The resulting `KTable` is materialized into another state store (additional to the provided state store names) - * as specified by the user via `Materialized` parameter, and is queryable through its given name. - * - * @param valueTransformerWithKeySupplier a instance of `ValueTransformerWithKeySupplier` that generates a `ValueTransformerWithKey` - * At least one transformer instance will be created per streaming task. - * Transformer implementations doe not need to be thread-safe. - * @param materialized an instance of `Materialized` used to describe how the state store of the - * resulting table should be materialized. - * @param stateStoreNames the names of the state stores used by the processor - * @return a [[KStream]] that contains records with unmodified key and new values (possibly of different type) - * @see `org.apache.kafka.streams.kstream.KStream#transformValues` - */ + * Create a new `KTable` by transforming the value of each record in this `KTable` into a new value, (with possibly new type). + * A `ValueTransformer` (provided by the given `ValueTransformerSupplier`) is applied to each input + * record value and computes a new value for it. + * This is similar to `#mapValues(ValueMapperWithKey)`, but more flexible, allowing stateful, rather than stateless, + * record-by-record operation, access to additional state-stores, and access to the `ProcessorContext`. + * In order to assign a state, the state must be created and registered + * beforehand via stores added via `addStateStore` or `addGlobalStore` before they can be connected to the `Transformer` + * The resulting `KTable` is materialized into another state store (additional to the provided state store names) + * as specified by the user via `Materialized` parameter, and is queryable through its given name. + * + * @param valueTransformerWithKeySupplier a instance of `ValueTransformerWithKeySupplier` that generates a `ValueTransformerWithKey` + * At least one transformer instance will be created per streaming task. + * Transformer implementations doe not need to be thread-safe. + * @param materialized an instance of `Materialized` used to describe how the state store of the + * resulting table should be materialized. + * @param stateStoreNames the names of the state stores used by the processor + * @return a [[KStream]] that contains records with unmodified key and new values (possibly of different type) + * @see `org.apache.kafka.streams.kstream.KStream#transformValues` + */ def transformValues[VR](valueTransformerWithKeySupplier: ValueTransformerWithKeySupplier[K, V, VR], materialized: Materialized[K, VR, KeyValueStore[Bytes, Array[Byte]]], - stateStoreNames: String*): KTable[K, VR] = { + stateStoreNames: String*): KTable[K, VR] = inner.transformValues[VR](valueTransformerWithKeySupplier, materialized, stateStoreNames: _*) - } /** * Re-groups the records of this [[KTable]] using the provided key/value mapper @@ -247,14 +241,13 @@ class KTable[K, V](val inner: KTableJ[K, V]) { * @param other the other [[KTable]] to be joined with this [[KTable]] * @param joiner a function that computes the join result for a pair of matching records * @param materialized a `Materialized` that describes how the `StateStore` for the resulting [[KTable]] - * should be materialized. + * should be materialized. * @return a [[KTable]] that contains join-records for each key and values computed by the given joiner, * one for each matched record-pair with the same key * @see `org.apache.kafka.streams.kstream.KTable#join` */ - def join[VO, VR](other: KTable[K, VO])( - joiner: (V, VO) => VR, - materialized: Materialized[K, VR, ByteArrayKeyValueStore] + def join[VO, VR](other: KTable[K, VO], materialized: Materialized[K, VR, ByteArrayKeyValueStore])( + joiner: (V, VO) => VR ): KTable[K, VR] = inner.join[VO, VR](other.inner, joiner.asValueJoiner, materialized) @@ -276,14 +269,13 @@ class KTable[K, V](val inner: KTableJ[K, V]) { * @param other the other [[KTable]] to be joined with this [[KTable]] * @param joiner a function that computes the join result for a pair of matching records * @param materialized a `Materialized` that describes how the `StateStore` for the resulting [[KTable]] - * should be materialized. + * should be materialized. * @return a [[KTable]] that contains join-records for each key and values computed by the given joiner, * one for each matched record-pair with the same key * @see `org.apache.kafka.streams.kstream.KTable#leftJoin` */ - def leftJoin[VO, VR](other: KTable[K, VO])( - joiner: (V, VO) => VR, - materialized: Materialized[K, VR, ByteArrayKeyValueStore] + def leftJoin[VO, VR](other: KTable[K, VO], materialized: Materialized[K, VR, ByteArrayKeyValueStore])( + joiner: (V, VO) => VR ): KTable[K, VR] = inner.leftJoin[VO, VR](other.inner, joiner.asValueJoiner, materialized) @@ -305,14 +297,13 @@ class KTable[K, V](val inner: KTableJ[K, V]) { * @param other the other [[KTable]] to be joined with this [[KTable]] * @param joiner a function that computes the join result for a pair of matching records * @param materialized a `Materialized` that describes how the `StateStore` for the resulting [[KTable]] - * should be materialized. + * should be materialized. * @return a [[KTable]] that contains join-records for each key and values computed by the given joiner, * one for each matched record-pair with the same key * @see `org.apache.kafka.streams.kstream.KTable#leftJoin` - */ - def outerJoin[VO, VR](other: KTable[K, VO])( - joiner: (V, VO) => VR, - materialized: Materialized[K, VR, ByteArrayKeyValueStore] + */ + def outerJoin[VO, VR](other: KTable[K, VO], materialized: Materialized[K, VR, ByteArrayKeyValueStore])( + joiner: (V, VO) => VR ): KTable[K, VR] = inner.outerJoin[VO, VR](other.inner, joiner.asValueJoiner, materialized) @@ -323,4 +314,3 @@ class KTable[K, V](val inner: KTableJ[K, V]) { */ def queryableStoreName: String = inner.queryableStoreName } - diff --git a/streams/streams-scala/src/main/scala/org/apache/kafka/streams/scala/kstream/SessionWindowedKStream.scala b/streams/streams-scala/src/main/scala/org/apache/kafka/streams/scala/kstream/SessionWindowedKStream.scala index ed41973c09080..a6027677edb90 100644 --- a/streams/streams-scala/src/main/scala/org/apache/kafka/streams/scala/kstream/SessionWindowedKStream.scala +++ b/streams/streams-scala/src/main/scala/org/apache/kafka/streams/scala/kstream/SessionWindowedKStream.scala @@ -46,8 +46,7 @@ class SessionWindowedKStream[K, V](val inner: SessionWindowedKStreamJ[K, V]) { * the latest (rolling) aggregate for each key within a window * @see `org.apache.kafka.streams.kstream.SessionWindowedKStream#aggregate` */ - def aggregate[VR](initializer: => VR)(aggregator: (K, V, VR) => VR, - merger: (K, VR, VR) => VR)( + def aggregate[VR](initializer: => VR)(aggregator: (K, V, VR) => VR, merger: (K, VR, VR) => VR)( implicit materialized: Materialized[K, VR, ByteArraySessionStore] ): KTable[Windowed[K], VR] = inner.aggregate((() => initializer).asInitializer, aggregator.asAggregator, merger.asMerger, materialized) @@ -55,7 +54,7 @@ class SessionWindowedKStream[K, V](val inner: SessionWindowedKStreamJ[K, V]) { /** * Count the number of records in this stream by the grouped key into `SessionWindows`. * - * @param materialized an instance of `Materialized` used to materialize a state store. + * @param materialized an instance of `Materialized` used to materialize a state store. * @return a windowed [[KTable]] that contains "update" records with unmodified keys and `Long` values * that represent the latest (rolling) count (i.e., number of records) for each key within a window * @see `org.apache.kafka.streams.kstream.SessionWindowedKStream#count` @@ -69,8 +68,8 @@ class SessionWindowedKStream[K, V](val inner: SessionWindowedKStreamJ[K, V]) { /** * Combine values of this stream by the grouped key into {@link SessionWindows}. * - * @param reducer a reducer function that computes a new aggregate result. - * @param materialized an instance of `Materialized` used to materialize a state store. + * @param reducer a reducer function that computes a new aggregate result. + * @param materialized an instance of `Materialized` used to materialize a state store. * @return a windowed [[KTable]] that contains "update" records with unmodified keys, and values that represent * the latest (rolling) aggregate for each key within a window * @see `org.apache.kafka.streams.kstream.SessionWindowedKStream#reduce` diff --git a/streams/streams-scala/src/main/scala/org/apache/kafka/streams/scala/kstream/TimeWindowedKStream.scala b/streams/streams-scala/src/main/scala/org/apache/kafka/streams/scala/kstream/TimeWindowedKStream.scala index 9e31ab9a80a4f..9be57949589f2 100644 --- a/streams/streams-scala/src/main/scala/org/apache/kafka/streams/scala/kstream/TimeWindowedKStream.scala +++ b/streams/streams-scala/src/main/scala/org/apache/kafka/streams/scala/kstream/TimeWindowedKStream.scala @@ -53,11 +53,11 @@ class TimeWindowedKStream[K, V](val inner: TimeWindowedKStreamJ[K, V]) { /** * Count the number of records in this stream by the grouped key and the defined windows. * - * @param materialized an instance of `Materialized` used to materialize a state store. + * @param materialized an instance of `Materialized` used to materialize a state store. * @return a [[KTable]] that contains "update" records with unmodified keys and `Long` values that * represent the latest (rolling) count (i.e., number of records) for each key * @see `org.apache.kafka.streams.kstream.TimeWindowedKStream#count` - */ + */ def count()(implicit materialized: Materialized[K, Long, ByteArrayWindowStore]): KTable[Windowed[K], Long] = { val c: KTable[Windowed[K], java.lang.Long] = inner.count(materialized.asInstanceOf[Materialized[K, java.lang.Long, ByteArrayWindowStore]]) @@ -68,7 +68,7 @@ class TimeWindowedKStream[K, V](val inner: TimeWindowedKStreamJ[K, V]) { * Combine the values of records in this stream by the grouped key. * * @param reducer a function that computes a new aggregate result - * @param materialized an instance of `Materialized` used to materialize a state store. + * @param materialized an instance of `Materialized` used to materialize a state store. * @return a [[KTable]] that contains "update" records with unmodified keys, and values that represent the * latest (rolling) aggregate for each key * @see `org.apache.kafka.streams.kstream.TimeWindowedKStream#reduce` diff --git a/streams/streams-scala/src/test/scala/org/apache/kafka/streams/scala/KStreamTest.scala b/streams/streams-scala/src/test/scala/org/apache/kafka/streams/scala/KStreamTest.scala new file mode 100644 index 0000000000000..3fdfee69fe3fe --- /dev/null +++ b/streams/streams-scala/src/test/scala/org/apache/kafka/streams/scala/KStreamTest.scala @@ -0,0 +1,157 @@ +/* + * Copyright (C) 2018 Joan Goyeau. + * + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF 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.scala + +import org.apache.kafka.streams.kstream.JoinWindows +import org.apache.kafka.streams.scala.Serdes._ +import org.apache.kafka.streams.scala.ImplicitConversions._ +import org.apache.kafka.streams.scala.utils.TestDriver +import org.junit.runner.RunWith +import org.scalatest.junit.JUnitRunner +import org.scalatest.{FlatSpec, Matchers} + +@RunWith(classOf[JUnitRunner]) +class KStreamTest extends FlatSpec with Matchers with TestDriver { + + "filter a KStream" should "filter records satisfying the predicate" in { + val builder = new StreamsBuilder() + val sourceTopic = "source" + val sinkTopic = "sink" + + builder.stream[String, String](sourceTopic).filter((_, value) => value != "value2").to(sinkTopic) + + val testDriver = createTestDriver(builder) + + testDriver.pipeRecord(sourceTopic, ("1", "value1")) + testDriver.readRecord[String, String](sinkTopic).value shouldBe "value1" + + testDriver.pipeRecord(sourceTopic, ("2", "value2")) + testDriver.readRecord[String, String](sinkTopic) shouldBe null + + testDriver.pipeRecord(sourceTopic, ("3", "value3")) + testDriver.readRecord[String, String](sinkTopic).value shouldBe "value3" + + testDriver.readRecord[String, String](sinkTopic) shouldBe null + + testDriver.close() + } + + "filterNot a KStream" should "filter records not satisfying the predicate" in { + val builder = new StreamsBuilder() + val sourceTopic = "source" + val sinkTopic = "sink" + + builder.stream[String, String](sourceTopic).filterNot((_, value) => value == "value2").to(sinkTopic) + + val testDriver = createTestDriver(builder) + + testDriver.pipeRecord(sourceTopic, ("1", "value1")) + testDriver.readRecord[String, String](sinkTopic).value shouldBe "value1" + + testDriver.pipeRecord(sourceTopic, ("2", "value2")) + testDriver.readRecord[String, String](sinkTopic) shouldBe null + + testDriver.pipeRecord(sourceTopic, ("3", "value3")) + testDriver.readRecord[String, String](sinkTopic).value shouldBe "value3" + + testDriver.readRecord[String, String](sinkTopic) shouldBe null + + testDriver.close() + } + + "foreach a KStream" should "run foreach actions on records" in { + val builder = new StreamsBuilder() + val sourceTopic = "source" + + var acc = "" + builder.stream[String, String](sourceTopic).foreach((_, value) => acc += value) + + val testDriver = createTestDriver(builder) + + testDriver.pipeRecord(sourceTopic, ("1", "value1")) + acc shouldBe "value1" + + testDriver.pipeRecord(sourceTopic, ("2", "value2")) + acc shouldBe "value1value2" + + testDriver.close() + } + + "peek a KStream" should "run peek actions on records" in { + val builder = new StreamsBuilder() + val sourceTopic = "source" + val sinkTopic = "sink" + + var acc = "" + builder.stream[String, String](sourceTopic).peek((k, v) => acc += v).to(sinkTopic) + + val testDriver = createTestDriver(builder) + + testDriver.pipeRecord(sourceTopic, ("1", "value1")) + acc shouldBe "value1" + testDriver.readRecord[String, String](sinkTopic).value shouldBe "value1" + + testDriver.pipeRecord(sourceTopic, ("2", "value2")) + acc shouldBe "value1value2" + testDriver.readRecord[String, String](sinkTopic).value shouldBe "value2" + + testDriver.close() + } + + "selectKey a KStream" should "select a new key" in { + val builder = new StreamsBuilder() + val sourceTopic = "source" + val sinkTopic = "sink" + + builder.stream[String, String](sourceTopic).selectKey((_, value) => value).to(sinkTopic) + + val testDriver = createTestDriver(builder) + + testDriver.pipeRecord(sourceTopic, ("1", "value1")) + testDriver.readRecord[String, String](sinkTopic).key shouldBe "value1" + + testDriver.pipeRecord(sourceTopic, ("1", "value2")) + testDriver.readRecord[String, String](sinkTopic).key shouldBe "value2" + + testDriver.readRecord[String, String](sinkTopic) shouldBe null + + testDriver.close() + } + + "join 2 KStreams" should "join correctly records" in { + val builder = new StreamsBuilder() + val sourceTopic1 = "source1" + val sourceTopic2 = "source2" + val sinkTopic = "sink" + + val stream1 = builder.stream[String, String](sourceTopic1) + val stream2 = builder.stream[String, String](sourceTopic2) + stream1.join(stream2)((a, b) => s"$a-$b", JoinWindows.of(1000)).to(sinkTopic) + + val testDriver = createTestDriver(builder) + + testDriver.pipeRecord(sourceTopic1, ("1", "topic1value1")) + testDriver.pipeRecord(sourceTopic2, ("1", "topic2value1")) + testDriver.readRecord[String, String](sinkTopic).value shouldBe "topic1value1-topic2value1" + + testDriver.readRecord[String, String](sinkTopic) shouldBe null + + testDriver.close() + } +} diff --git a/streams/streams-scala/src/test/scala/org/apache/kafka/streams/scala/KTableTest.scala b/streams/streams-scala/src/test/scala/org/apache/kafka/streams/scala/KTableTest.scala new file mode 100644 index 0000000000000..2e9c821ed800c --- /dev/null +++ b/streams/streams-scala/src/test/scala/org/apache/kafka/streams/scala/KTableTest.scala @@ -0,0 +1,145 @@ +/* + * Copyright (C) 2018 Joan Goyeau. + * + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF 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.scala + +import org.apache.kafka.streams.kstream.Materialized +import org.apache.kafka.streams.scala.ImplicitConversions._ +import org.apache.kafka.streams.scala.Serdes._ +import org.apache.kafka.streams.scala.utils.TestDriver +import org.junit.runner.RunWith +import org.scalatest.junit.JUnitRunner +import org.scalatest.{FlatSpec, Matchers} + +@RunWith(classOf[JUnitRunner]) +class KTableTest extends FlatSpec with Matchers with TestDriver { + + "filter a KTable" should "filter records satisfying the predicate" in { + val builder = new StreamsBuilder() + val sourceTopic = "source" + val sinkTopic = "sink" + + val table = builder.stream[String, String](sourceTopic).groupBy((key, _) => key).count() + table.filter((_, value) => value > 1).toStream.to(sinkTopic) + + val testDriver = createTestDriver(builder) + + { + testDriver.pipeRecord(sourceTopic, ("1", "value1")) + val record = testDriver.readRecord[String, Long](sinkTopic) + record.key shouldBe "1" + record.value shouldBe (null: java.lang.Long) + } + { + testDriver.pipeRecord(sourceTopic, ("1", "value2")) + val record = testDriver.readRecord[String, Long](sinkTopic) + record.key shouldBe "1" + record.value shouldBe 2 + } + { + testDriver.pipeRecord(sourceTopic, ("2", "value1")) + val record = testDriver.readRecord[String, Long](sinkTopic) + record.key shouldBe "2" + record.value shouldBe (null: java.lang.Long) + } + testDriver.readRecord[String, Long](sinkTopic) shouldBe null + + testDriver.close() + } + + "filterNot a KTable" should "filter records not satisfying the predicate" in { + val builder = new StreamsBuilder() + val sourceTopic = "source" + val sinkTopic = "sink" + + val table = builder.stream[String, String](sourceTopic).groupBy((key, _) => key).count() + table.filterNot((_, value) => value > 1).toStream.to(sinkTopic) + + val testDriver = createTestDriver(builder) + + { + testDriver.pipeRecord(sourceTopic, ("1", "value1")) + val record = testDriver.readRecord[String, Long](sinkTopic) + record.key shouldBe "1" + record.value shouldBe 1 + } + { + testDriver.pipeRecord(sourceTopic, ("1", "value2")) + val record = testDriver.readRecord[String, Long](sinkTopic) + record.key shouldBe "1" + record.value shouldBe (null: java.lang.Long) + } + { + testDriver.pipeRecord(sourceTopic, ("2", "value1")) + val record = testDriver.readRecord[String, Long](sinkTopic) + record.key shouldBe "2" + record.value shouldBe 1 + } + testDriver.readRecord[String, Long](sinkTopic) shouldBe null + + testDriver.close() + } + + "join 2 KTables" should "join correctly records" in { + val builder = new StreamsBuilder() + val sourceTopic1 = "source1" + val sourceTopic2 = "source2" + val sinkTopic = "sink" + + val table1 = builder.stream[String, String](sourceTopic1).groupBy((key, _) => key).count() + val table2 = builder.stream[String, String](sourceTopic2).groupBy((key, _) => key).count() + table1.join(table2)((a, b) => a + b).toStream.to(sinkTopic) + + val testDriver = createTestDriver(builder) + + testDriver.pipeRecord(sourceTopic1, ("1", "topic1value1")) + testDriver.pipeRecord(sourceTopic2, ("1", "topic2value1")) + testDriver.readRecord[String, Long](sinkTopic).value shouldBe 2 + + testDriver.readRecord[String, Long](sinkTopic) shouldBe null + + testDriver.close() + } + + "join 2 KTables with a Materialized" should "join correctly records and state store" in { + val builder = new StreamsBuilder() + val sourceTopic1 = "source1" + val sourceTopic2 = "source2" + val sinkTopic = "sink" + val stateStore = "store" + val materialized = Materialized + .as[String, Long, ByteArrayKeyValueStore](stateStore) + .withKeySerde(Serdes.String) + .withValueSerde(Serdes.Long) + + val table1 = builder.stream[String, String](sourceTopic1).groupBy((key, _) => key).count() + val table2 = builder.stream[String, String](sourceTopic2).groupBy((key, _) => key).count() + table1.join(table2, materialized)((a, b) => a + b).toStream.to(sinkTopic) + + val testDriver = createTestDriver(builder) + + testDriver.pipeRecord(sourceTopic1, ("1", "topic1value1")) + testDriver.pipeRecord(sourceTopic2, ("1", "topic2value1")) + testDriver.readRecord[String, Long](sinkTopic).value shouldBe 2 + testDriver.getKeyValueStore[String, Long](stateStore).get("1") shouldBe 2 + + testDriver.readRecord[String, Long](sinkTopic) shouldBe null + + testDriver.close() + } +} diff --git a/streams/streams-scala/src/test/scala/org/apache/kafka/streams/scala/StreamToTableJoinScalaIntegrationTestImplicitSerdes.scala b/streams/streams-scala/src/test/scala/org/apache/kafka/streams/scala/StreamToTableJoinScalaIntegrationTestImplicitSerdes.scala index 02d1dabb9cf19..91634bec29437 100644 --- a/streams/streams-scala/src/test/scala/org/apache/kafka/streams/scala/StreamToTableJoinScalaIntegrationTestImplicitSerdes.scala +++ b/streams/streams-scala/src/test/scala/org/apache/kafka/streams/scala/StreamToTableJoinScalaIntegrationTestImplicitSerdes.scala @@ -18,16 +18,16 @@ package org.apache.kafka.streams.scala import java.util.Properties +import kafka.utils.MockTime import org.apache.kafka.clients.consumer.ConsumerConfig import org.apache.kafka.clients.producer.ProducerConfig -import org.apache.kafka.common.serialization._ -import org.apache.kafka.common.utils.MockTime +import org.apache.kafka.common.serialization.{LongDeserializer, LongSerializer, StringDeserializer, StringSerializer} import org.apache.kafka.streams._ import org.apache.kafka.streams.integration.utils.{EmbeddedKafkaCluster, IntegrationTestUtils} import org.apache.kafka.streams.scala.ImplicitConversions._ import org.apache.kafka.streams.scala.kstream._ +import org.apache.kafka.streams.scala.utils.StreamToTableJoinTestData import org.apache.kafka.test.TestUtils -import org.junit.Assert._ import org.junit._ import org.junit.rules.TemporaryFolder import org.scalatest.junit.JUnitSuite @@ -40,9 +40,8 @@ import org.scalatest.junit.JUnitSuite *

                        * Note: In the current project settings SAM type conversion is turned off as it's experimental in Scala 2.11. * Hence the native Java API based version is more verbose. - */ -class StreamToTableJoinScalaIntegrationTestImplicitSerdes extends JUnitSuite - with StreamToTableJoinTestData { + */ +class StreamToTableJoinScalaIntegrationTestImplicitSerdes extends JUnitSuite with StreamToTableJoinTestData { private val privateCluster: EmbeddedKafkaCluster = new EmbeddedKafkaCluster(1) @@ -67,7 +66,7 @@ class StreamToTableJoinScalaIntegrationTestImplicitSerdes extends JUnitSuite @Test def testShouldCountClicksPerRegion(): Unit = { - // DefaultSerdes brings into scope implicit serdes (mostly for primitives) that will set up all Serialized, Produced, + // DefaultSerdes brings into scope implicit serdes (mostly for primitives) that will set up all Serialized, Produced, // Consumed and Joined instances. So all APIs below that accept Serialized, Produced, Consumed or Joined will // get these instances automatically import Serdes._ @@ -84,7 +83,7 @@ class StreamToTableJoinScalaIntegrationTestImplicitSerdes extends JUnitSuite val clicksPerRegion: KTable[String, Long] = userClicksStream - // Join the stream against the table. + // Join the stream against the table. .leftJoin(userRegionsTable)((clicks, region) => (if (region == null) "UNKNOWN" else region, clicks)) // Change the stream from -> to -> @@ -100,14 +99,10 @@ class StreamToTableJoinScalaIntegrationTestImplicitSerdes extends JUnitSuite val streams: KafkaStreams = new KafkaStreams(builder.build(), streamsConfiguration) streams.start() - - val actualClicksPerRegion: java.util.List[KeyValue[String, Long]] = + val actualClicksPerRegion: java.util.List[KeyValue[String, Long]] = produceNConsume(userClicksTopic, userRegionsTopic, outputTopic) streams.close() - - import collection.JavaConverters._ - assertEquals(actualClicksPerRegion.asScala.sortBy(_.key), expectedClicksPerRegion.sortBy(_.key)) } @Test def testShouldCountClicksPerRegionJava(): Unit = { @@ -126,29 +121,32 @@ class StreamToTableJoinScalaIntegrationTestImplicitSerdes extends JUnitSuite val builder: StreamsBuilderJ = new StreamsBuilderJ() - val userClicksStream: KStreamJ[String, JLong] = + val userClicksStream: KStreamJ[String, JLong] = builder.stream[String, JLong](userClicksTopicJ, Consumed.`with`(Serdes.String, Serdes.JavaLong)) - val userRegionsTable: KTableJ[String, String] = + val userRegionsTable: KTableJ[String, String] = builder.table[String, String](userRegionsTopicJ, Consumed.`with`(Serdes.String, Serdes.String)) // Join the stream against the table. val userClicksJoinRegion: KStreamJ[String, (String, JLong)] = userClicksStream - .leftJoin(userRegionsTable, + .leftJoin( + userRegionsTable, new ValueJoiner[JLong, String, (String, JLong)] { - def apply(clicks: JLong, region: String): (String, JLong) = + def apply(clicks: JLong, region: String): (String, JLong) = (if (region == null) "UNKNOWN" else region, clicks) - }, - Joined.`with`[String, JLong, String](Serdes.String, Serdes.JavaLong, Serdes.String)) + }, + Joined.`with`[String, JLong, String](Serdes.String, Serdes.JavaLong, Serdes.String) + ) // Change the stream from -> to -> - val clicksByRegion : KStreamJ[String, JLong] = userClicksJoinRegion - .map { + val clicksByRegion: KStreamJ[String, JLong] = userClicksJoinRegion + .map { new KeyValueMapper[String, (String, JLong), KeyValue[String, JLong]] { - def apply(k: String, regionWithClicks: (String, JLong)) = new KeyValue[String, JLong](regionWithClicks._1, regionWithClicks._2) + def apply(k: String, regionWithClicks: (String, JLong)) = + new KeyValue[String, JLong](regionWithClicks._1, regionWithClicks._2) } } - + // Compute the total per region by summing the individual click counts per region. val clicksPerRegion: KTableJ[String, JLong] = clicksByRegion .groupByKey(Serialized.`with`(Serdes.String, Serdes.JavaLong)) @@ -157,19 +155,15 @@ class StreamToTableJoinScalaIntegrationTestImplicitSerdes extends JUnitSuite def apply(v1: JLong, v2: JLong) = v1 + v2 } } - + // Write the (continuously updating) results to the output topic. clicksPerRegion.toStream.to(outputTopicJ, Produced.`with`(Serdes.String, Serdes.JavaLong)) val streams: KafkaStreamsJ = new KafkaStreamsJ(builder.build(), streamsConfiguration) streams.start() - - val actualClicksPerRegion: java.util.List[KeyValue[String, Long]] = - produceNConsume(userClicksTopicJ, userRegionsTopicJ, outputTopicJ) - + produceNConsume(userClicksTopicJ, userRegionsTopicJ, outputTopicJ) streams.close() - assertEquals(actualClicksPerRegion.asScala.sortBy(_.key), expectedClicksPerRegion.sortBy(_.key)) } private def getStreamsConfiguration(): Properties = { @@ -214,17 +208,27 @@ class StreamToTableJoinScalaIntegrationTestImplicitSerdes extends JUnitSuite p } - private def produceNConsume(userClicksTopic: String, userRegionsTopic: String, outputTopic: String): java.util.List[KeyValue[String, Long]] = { + private def produceNConsume(userClicksTopic: String, + userRegionsTopic: String, + outputTopic: String): java.util.List[KeyValue[String, Long]] = { import collection.JavaConverters._ - + // Publish user-region information. val userRegionsProducerConfig: Properties = getUserRegionsProducerConfig() - IntegrationTestUtils.produceKeyValuesSynchronously(userRegionsTopic, userRegions.asJava, userRegionsProducerConfig, mockTime, false) + IntegrationTestUtils.produceKeyValuesSynchronously(userRegionsTopic, + userRegions.asJava, + userRegionsProducerConfig, + mockTime, + false) // Publish user-click information. val userClicksProducerConfig: Properties = getUserClicksProducerConfig() - IntegrationTestUtils.produceKeyValuesSynchronously(userClicksTopic, userClicks.asJava, userClicksProducerConfig, mockTime, false) + IntegrationTestUtils.produceKeyValuesSynchronously(userClicksTopic, + userClicks.asJava, + userClicksProducerConfig, + mockTime, + false) // consume and verify result val consumerConfig = getConsumerConfig() @@ -232,4 +236,3 @@ class StreamToTableJoinScalaIntegrationTestImplicitSerdes extends JUnitSuite IntegrationTestUtils.waitUntilMinKeyValueRecordsReceived(consumerConfig, outputTopic, expectedClicksPerRegion.size) } } - diff --git a/streams/streams-scala/src/test/scala/org/apache/kafka/streams/scala/TopologyTest.scala b/streams/streams-scala/src/test/scala/org/apache/kafka/streams/scala/TopologyTest.scala index ffae666fcf127..b596dd37fa684 100644 --- a/streams/streams-scala/src/test/scala/org/apache/kafka/streams/scala/TopologyTest.scala +++ b/streams/streams-scala/src/test/scala/org/apache/kafka/streams/scala/TopologyTest.scala @@ -21,21 +21,20 @@ package org.apache.kafka.streams.scala import java.util.regex.Pattern -import org.scalatest.junit.JUnitSuite +import org.apache.kafka.streams.kstream.{KGroupedStream => KGroupedStreamJ, KStream => KStreamJ, KTable => KTableJ, _} +import org.apache.kafka.streams.processor.ProcessorContext +import org.apache.kafka.streams.scala.ImplicitConversions._ +import org.apache.kafka.streams.scala.kstream._ +import org.apache.kafka.streams.{StreamsBuilder => StreamsBuilderJ, _} import org.junit.Assert._ import org.junit._ +import org.scalatest.junit.JUnitSuite -import org.apache.kafka.streams.scala.kstream._ - -import ImplicitConversions._ - -import org.apache.kafka.streams.{StreamsBuilder => StreamsBuilderJ, _} -import org.apache.kafka.streams.kstream.{KTable => KTableJ, KStream => KStreamJ, KGroupedStream => KGroupedStreamJ, _} -import collection.JavaConverters._ +import _root_.scala.collection.JavaConverters._ /** * Test suite that verifies that the topology built by the Java and Scala APIs match. - */ + */ class TopologyTest extends JUnitSuite { val inputTopic = "input-topic" @@ -50,22 +49,22 @@ class TopologyTest extends JUnitSuite { def getTopologyScala(): TopologyDescription = { import Serdes._ - + val streamBuilder = new StreamsBuilder val textLines = streamBuilder.stream[String, String](inputTopic) - + val _: KStream[String, String] = textLines.flatMapValues(v => pattern.split(v.toLowerCase)) - + streamBuilder.build().describe() } - + // build the Java topology def getTopologyJava(): TopologyDescription = { val streamBuilder = new StreamsBuilderJ val textLines = streamBuilder.stream[String, String](inputTopic) - + val _: KStreamJ[String, String] = textLines.flatMapValues { new ValueMapper[String, java.lang.Iterable[String]] { def apply(s: String): java.lang.Iterable[String] = pattern.split(s.toLowerCase).toIterable.asJava @@ -84,15 +83,16 @@ class TopologyTest extends JUnitSuite { def getTopologyScala(): TopologyDescription = { import Serdes._ - + val streamBuilder = new StreamsBuilder val textLines = streamBuilder.stream[String, String](inputTopic) - + val _: KTable[String, Long] = - textLines.flatMapValues(v => pattern.split(v.toLowerCase)) + textLines + .flatMapValues(v => pattern.split(v.toLowerCase)) .groupBy((k, v) => v) .count() - + streamBuilder.build().describe() } @@ -101,21 +101,21 @@ class TopologyTest extends JUnitSuite { val streamBuilder = new StreamsBuilderJ val textLines: KStreamJ[String, String] = streamBuilder.stream[String, String](inputTopic) - + val splits: KStreamJ[String, String] = textLines.flatMapValues { new ValueMapper[String, java.lang.Iterable[String]] { def apply(s: String): java.lang.Iterable[String] = pattern.split(s.toLowerCase).toIterable.asJava } } - + val grouped: KGroupedStreamJ[String, String] = splits.groupBy { new KeyValueMapper[String, String, String] { def apply(k: String, v: String): String = v } } - + val wordCounts: KTableJ[String, java.lang.Long] = grouped.count() - + streamBuilder.build().describe() } @@ -128,13 +128,13 @@ class TopologyTest extends JUnitSuite { // build the Scala topology def getTopologyScala(): TopologyDescription = { import Serdes._ - + val builder = new StreamsBuilder() - + val userClicksStream: KStream[String, Long] = builder.stream(userClicksTopic) - + val userRegionsTable: KTable[String, String] = builder.table(userRegionsTopic) - + val clicksPerRegion: KTable[String, Long] = userClicksStream .leftJoin(userRegionsTable)((clicks, region) => (if (region == null) "UNKNOWN" else region, clicks)) @@ -149,32 +149,35 @@ class TopologyTest extends JUnitSuite { def getTopologyJava(): TopologyDescription = { import java.lang.{Long => JLong} - + val builder: StreamsBuilderJ = new StreamsBuilderJ() - - val userClicksStream: KStreamJ[String, JLong] = + + val userClicksStream: KStreamJ[String, JLong] = builder.stream[String, JLong](userClicksTopic, Consumed.`with`(Serdes.String, Serdes.JavaLong)) - - val userRegionsTable: KTableJ[String, String] = + + val userRegionsTable: KTableJ[String, String] = builder.table[String, String](userRegionsTopic, Consumed.`with`(Serdes.String, Serdes.String)) - + // Join the stream against the table. val userClicksJoinRegion: KStreamJ[String, (String, JLong)] = userClicksStream - .leftJoin(userRegionsTable, + .leftJoin( + userRegionsTable, new ValueJoiner[JLong, String, (String, JLong)] { - def apply(clicks: JLong, region: String): (String, JLong) = + def apply(clicks: JLong, region: String): (String, JLong) = (if (region == null) "UNKNOWN" else region, clicks) - }, - Joined.`with`[String, JLong, String](Serdes.String, Serdes.JavaLong, Serdes.String)) - + }, + Joined.`with`[String, JLong, String](Serdes.String, Serdes.JavaLong, Serdes.String) + ) + // Change the stream from -> to -> - val clicksByRegion : KStreamJ[String, JLong] = userClicksJoinRegion - .map { + val clicksByRegion: KStreamJ[String, JLong] = userClicksJoinRegion + .map { new KeyValueMapper[String, (String, JLong), KeyValue[String, JLong]] { - def apply(k: String, regionWithClicks: (String, JLong)) = new KeyValue[String, JLong](regionWithClicks._1, regionWithClicks._2) + def apply(k: String, regionWithClicks: (String, JLong)) = + new KeyValue[String, JLong](regionWithClicks._1, regionWithClicks._2) } } - + // Compute the total per region by summing the individual click counts per region. val clicksPerRegion: KTableJ[String, JLong] = clicksByRegion .groupByKey(Serialized.`with`(Serdes.String, Serdes.JavaLong)) @@ -190,4 +193,68 @@ class TopologyTest extends JUnitSuite { // should match assertEquals(getTopologyScala(), getTopologyJava()) } + + @Test def shouldBuildIdenticalTopologyInJavaNScalaTransform() = { + + // build the Scala topology + def getTopologyScala(): TopologyDescription = { + + import Serdes._ + + val streamBuilder = new StreamsBuilder + val textLines = streamBuilder.stream[String, String](inputTopic) + + //noinspection ConvertExpressionToSAM due to 2.11 build + val _: KTable[String, Long] = + textLines + .transform(new TransformerSupplier[String, String, KeyValue[String, String]] { + override def get(): Transformer[String, String, KeyValue[String, String]] = + new Transformer[String, String, KeyValue[String, String]] { + override def init(context: ProcessorContext): Unit = Unit + + override def transform(key: String, value: String): KeyValue[String, String] = + new KeyValue(key, value.toLowerCase) + + override def close(): Unit = Unit + } + }) + .groupBy((k, v) => v) + .count() + + streamBuilder.build().describe() + } + + // build the Java topology + def getTopologyJava(): TopologyDescription = { + + val streamBuilder = new StreamsBuilderJ + val textLines: KStreamJ[String, String] = streamBuilder.stream[String, String](inputTopic) + + val lowered: KStreamJ[String, String] = textLines + .transform(new TransformerSupplier[String, String, KeyValue[String, String]] { + override def get(): Transformer[String, String, KeyValue[String, String]] = + new Transformer[String, String, KeyValue[String, String]] { + override def init(context: ProcessorContext): Unit = Unit + + override def transform(key: String, value: String): KeyValue[String, String] = + new KeyValue(key, value.toLowerCase) + + override def close(): Unit = Unit + } + }) + + val grouped: KGroupedStreamJ[String, String] = lowered.groupBy { + new KeyValueMapper[String, String, String] { + def apply(k: String, v: String): String = v + } + } + + val wordCounts: KTableJ[String, java.lang.Long] = grouped.count() + + streamBuilder.build().describe() + } + + // should match + assertEquals(getTopologyScala(), getTopologyJava()) + } } diff --git a/streams/streams-scala/src/test/scala/org/apache/kafka/streams/scala/WordCountTest.scala b/streams/streams-scala/src/test/scala/org/apache/kafka/streams/scala/WordCountTest.scala index 5abc1bcf6ffc9..5d858d8781ffb 100644 --- a/streams/streams-scala/src/test/scala/org/apache/kafka/streams/scala/WordCountTest.scala +++ b/streams/streams-scala/src/test/scala/org/apache/kafka/streams/scala/WordCountTest.scala @@ -50,7 +50,7 @@ import ImplicitConversions._ *

                        * Note: In the current project settings SAM type conversion is turned off as it's experimental in Scala 2.11. * Hence the native Java API based version is more verbose. - */ + */ class WordCountTest extends JUnitSuite with WordCountTestData { private val privateCluster: EmbeddedKafkaCluster = new EmbeddedKafkaCluster(1) @@ -61,11 +61,8 @@ class WordCountTest extends JUnitSuite with WordCountTestData { val mockTime: MockTime = cluster.time mockTime.setCurrentTimeMs(alignedTime) - val tFolder: TemporaryFolder = new TemporaryFolder(TestUtils.tempDirectory()) @Rule def testFolder: TemporaryFolder = tFolder - - @Before def startKafkaCluster(): Unit = { cluster.createTopic(inputTopic) @@ -86,7 +83,8 @@ class WordCountTest extends JUnitSuite with WordCountTestData { // generate word counts val wordCounts: KTable[String, Long] = - textLines.flatMapValues(v => pattern.split(v.toLowerCase)) + textLines + .flatMapValues(v => pattern.split(v.toLowerCase)) .groupBy((_, v) => v) .count() @@ -117,7 +115,8 @@ class WordCountTest extends JUnitSuite with WordCountTestData { // generate word counts val wordCounts: KTable[String, Long] = - textLines.flatMapValues(v => pattern.split(v.toLowerCase)) + textLines + .flatMapValues(v => pattern.split(v.toLowerCase)) .groupBy((k, v) => v) .count()(Materialized.as("word-count")) @@ -139,7 +138,12 @@ class WordCountTest extends JUnitSuite with WordCountTestData { @Test def testShouldCountWordsJava(): Unit = { import org.apache.kafka.streams.{KafkaStreams => KafkaStreamsJ, StreamsBuilder => StreamsBuilderJ} - import org.apache.kafka.streams.kstream.{KTable => KTableJ, KStream => KStreamJ, KGroupedStream => KGroupedStreamJ, _} + import org.apache.kafka.streams.kstream.{ + KTable => KTableJ, + KStream => KStreamJ, + KGroupedStream => KGroupedStreamJ, + _ + } import collection.JavaConverters._ val streamsConfiguration = getStreamsConfiguration() @@ -250,4 +254,3 @@ trait WordCountTestData { new KeyValue("слова", 1L) ) } - diff --git a/streams/streams-scala/src/test/scala/org/apache/kafka/streams/scala/utils/StreamToTableJoinScalaIntegrationTestBase.scala b/streams/streams-scala/src/test/scala/org/apache/kafka/streams/scala/utils/StreamToTableJoinScalaIntegrationTestBase.scala new file mode 100644 index 0000000000000..9a3ee7f27942e --- /dev/null +++ b/streams/streams-scala/src/test/scala/org/apache/kafka/streams/scala/utils/StreamToTableJoinScalaIntegrationTestBase.scala @@ -0,0 +1,135 @@ +/* + * Copyright (C) 2018 Lightbend Inc. + * Copyright (C) 2017-2018 Alexis Seigneurin. + * + * 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 org.apache.kafka.streams.scala.utils + +import java.util.Properties + +import org.apache.kafka.clients.consumer.ConsumerConfig +import org.apache.kafka.clients.producer.ProducerConfig +import org.apache.kafka.common.serialization._ +import org.apache.kafka.common.utils.MockTime +import org.apache.kafka.streams._ +import org.apache.kafka.streams.integration.utils.{EmbeddedKafkaCluster, IntegrationTestUtils} +import org.apache.kafka.test.TestUtils +import org.junit._ +import org.junit.rules.TemporaryFolder +import org.scalatest.junit.JUnitSuite + +/** + * Test suite base that prepares Kafka cluster for stream-table joins in Kafka Streams + *

                        + */ +class StreamToTableJoinScalaIntegrationTestBase extends JUnitSuite with StreamToTableJoinTestData { + + private val privateCluster: EmbeddedKafkaCluster = new EmbeddedKafkaCluster(1) + + @Rule def cluster: EmbeddedKafkaCluster = privateCluster + + final val alignedTime = (System.currentTimeMillis() / 1000 + 1) * 1000 + val mockTime: MockTime = cluster.time + mockTime.setCurrentTimeMs(alignedTime) + + val tFolder: TemporaryFolder = new TemporaryFolder(TestUtils.tempDirectory()) + @Rule def testFolder: TemporaryFolder = tFolder + + @Before + def startKafkaCluster(): Unit = { + cluster.createTopic(userClicksTopic) + cluster.createTopic(userRegionsTopic) + cluster.createTopic(outputTopic) + cluster.createTopic(userClicksTopicJ) + cluster.createTopic(userRegionsTopicJ) + cluster.createTopic(outputTopicJ) + } + + def getStreamsConfiguration(): Properties = { + val streamsConfiguration: Properties = new Properties() + + streamsConfiguration.put(StreamsConfig.APPLICATION_ID_CONFIG, "stream-table-join-scala-integration-test") + streamsConfiguration.put(StreamsConfig.BOOTSTRAP_SERVERS_CONFIG, cluster.bootstrapServers()) + streamsConfiguration.put(StreamsConfig.COMMIT_INTERVAL_MS_CONFIG, "1000") + streamsConfiguration.put(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG, "earliest") + streamsConfiguration.put(StreamsConfig.STATE_DIR_CONFIG, testFolder.getRoot.getPath) + + streamsConfiguration + } + + private def getUserRegionsProducerConfig(): Properties = { + val p = new Properties() + p.put(ProducerConfig.BOOTSTRAP_SERVERS_CONFIG, cluster.bootstrapServers()) + p.put(ProducerConfig.ACKS_CONFIG, "all") + p.put(ProducerConfig.RETRIES_CONFIG, "0") + p.put(ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG, classOf[StringSerializer]) + p.put(ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG, classOf[StringSerializer]) + p + } + + private def getUserClicksProducerConfig(): Properties = { + val p = new Properties() + p.put(ProducerConfig.BOOTSTRAP_SERVERS_CONFIG, cluster.bootstrapServers()) + p.put(ProducerConfig.ACKS_CONFIG, "all") + p.put(ProducerConfig.RETRIES_CONFIG, "0") + p.put(ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG, classOf[StringSerializer]) + p.put(ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG, classOf[LongSerializer]) + p + } + + private def getConsumerConfig(): Properties = { + val p = new Properties() + p.put(ConsumerConfig.BOOTSTRAP_SERVERS_CONFIG, cluster.bootstrapServers()) + p.put(ConsumerConfig.GROUP_ID_CONFIG, "join-scala-integration-test-standard-consumer") + p.put(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG, "earliest") + p.put(ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG, classOf[StringDeserializer]) + p.put(ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG, classOf[LongDeserializer]) + p + } + + def produceNConsume(userClicksTopic: String, + userRegionsTopic: String, + outputTopic: String, + waitTillRecordsReceived: Boolean = true): java.util.List[KeyValue[String, Long]] = { + + import collection.JavaConverters._ + + // Publish user-region information. + val userRegionsProducerConfig: Properties = getUserRegionsProducerConfig() + IntegrationTestUtils.produceKeyValuesSynchronously(userRegionsTopic, + userRegions.asJava, + userRegionsProducerConfig, + mockTime, + false) + + // Publish user-click information. + val userClicksProducerConfig: Properties = getUserClicksProducerConfig() + IntegrationTestUtils.produceKeyValuesSynchronously(userClicksTopic, + userClicks.asJava, + userClicksProducerConfig, + mockTime, + false) + + if (waitTillRecordsReceived) { + // consume and verify result + val consumerConfig = getConsumerConfig() + + IntegrationTestUtils.waitUntilFinalKeyValueRecordsReceived(consumerConfig, + outputTopic, + expectedClicksPerRegion.asJava) + } else { + java.util.Collections.emptyList() + } + } +} diff --git a/streams/streams-scala/src/test/scala/org/apache/kafka/streams/scala/StreamToTableJoinTestData.scala b/streams/streams-scala/src/test/scala/org/apache/kafka/streams/scala/utils/StreamToTableJoinTestData.scala similarity index 97% rename from streams/streams-scala/src/test/scala/org/apache/kafka/streams/scala/StreamToTableJoinTestData.scala rename to streams/streams-scala/src/test/scala/org/apache/kafka/streams/scala/utils/StreamToTableJoinTestData.scala index 45715a7abe6a5..890d8c2ee14b2 100644 --- a/streams/streams-scala/src/test/scala/org/apache/kafka/streams/scala/StreamToTableJoinTestData.scala +++ b/streams/streams-scala/src/test/scala/org/apache/kafka/streams/scala/utils/StreamToTableJoinTestData.scala @@ -14,7 +14,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.kafka.streams.scala +package org.apache.kafka.streams.scala.utils import org.apache.kafka.streams.KeyValue @@ -58,4 +58,3 @@ trait StreamToTableJoinTestData { new KeyValue("asia", 124L) ) } - diff --git a/streams/streams-scala/src/test/scala/org/apache/kafka/streams/scala/utils/TestDriver.scala b/streams/streams-scala/src/test/scala/org/apache/kafka/streams/scala/utils/TestDriver.scala new file mode 100644 index 0000000000000..1497dd747931d --- /dev/null +++ b/streams/streams-scala/src/test/scala/org/apache/kafka/streams/scala/utils/TestDriver.scala @@ -0,0 +1,52 @@ +/* + * Copyright (C) 2018 Joan Goyeau. + * + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF 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.scala.utils + +import java.util.{Properties, UUID} + +import org.apache.kafka.clients.producer.ProducerRecord +import org.apache.kafka.common.serialization.Serde +import org.apache.kafka.streams.scala.StreamsBuilder +import org.apache.kafka.streams.test.ConsumerRecordFactory +import org.apache.kafka.streams.{StreamsConfig, TopologyTestDriver} +import org.scalatest.Suite + +trait TestDriver { this: Suite => + + def createTestDriver(builder: StreamsBuilder, initialWallClockTimeMs: Long = System.currentTimeMillis()) = { + val config = new Properties() + config.put(StreamsConfig.APPLICATION_ID_CONFIG, "test") + config.put(StreamsConfig.BOOTSTRAP_SERVERS_CONFIG, "dummy:1234") + config.put(StreamsConfig.STATE_DIR_CONFIG, s"out/state-store-${UUID.randomUUID()}") + new TopologyTestDriver(builder.build(), config, initialWallClockTimeMs) + } + + implicit class TopologyTestDriverOps(inner: TopologyTestDriver) { + def pipeRecord[K, V](topic: String, record: (K, V), timestampMs: Long = System.currentTimeMillis())( + implicit serdeKey: Serde[K], + serdeValue: Serde[V] + ): Unit = { + val recordFactory = new ConsumerRecordFactory[K, V](serdeKey.serializer, serdeValue.serializer) + inner.pipeInput(recordFactory.create(topic, record._1, record._2, timestampMs)) + } + + def readRecord[K, V](topic: String)(implicit serdeKey: Serde[K], serdeValue: Serde[V]): ProducerRecord[K, V] = + inner.readOutput(topic, serdeKey.deserializer, serdeValue.deserializer) + } +} diff --git a/streams/test-utils/src/main/java/org/apache/kafka/streams/TopologyTestDriver.java b/streams/test-utils/src/main/java/org/apache/kafka/streams/TopologyTestDriver.java index e46ec6a35d0f7..96904c2d2e1e2 100644 --- a/streams/test-utils/src/main/java/org/apache/kafka/streams/TopologyTestDriver.java +++ b/streams/test-utils/src/main/java/org/apache/kafka/streams/TopologyTestDriver.java @@ -28,6 +28,7 @@ import org.apache.kafka.common.PartitionInfo; import org.apache.kafka.common.TopicPartition; import org.apache.kafka.common.annotation.InterfaceStability; +import org.apache.kafka.common.header.internals.RecordHeaders; import org.apache.kafka.common.metrics.Metrics; import org.apache.kafka.common.record.TimestampType; import org.apache.kafka.common.serialization.ByteArraySerializer; @@ -65,6 +66,7 @@ import java.io.Closeable; import java.io.IOException; +import java.time.Duration; import java.util.ArrayList; import java.util.Collection; import java.util.Collections; @@ -170,18 +172,21 @@ @InterfaceStability.Evolving public class TopologyTestDriver implements Closeable { - private final Time mockTime; + private final Time mockWallClockTime; private final InternalTopologyBuilder internalTopologyBuilder; private final static int PARTITION_ID = 0; private final static TaskId TASK_ID = new TaskId(0, PARTITION_ID); - private final StreamTask task; + final StreamTask task; private final GlobalStateUpdateTask globalStateTask; private final GlobalStateManager globalStateManager; + private final InternalProcessorContext context; + private final StateDirectory stateDirectory; private final Metrics metrics; - private final ProcessorTopology processorTopology; + final ProcessorTopology processorTopology; + final ProcessorTopology globalTopology; private final MockProducer producer; @@ -191,6 +196,7 @@ public class TopologyTestDriver implements Closeable { private final Map offsetsByTopicPartition = new HashMap<>(); private final Map>> outputRecordsByTopic = new HashMap<>(); + private final boolean eosEnabled; /** * Create a new test diver instance. @@ -216,22 +222,9 @@ public TopologyTestDriver(final Topology topology, public TopologyTestDriver(final Topology topology, final Properties config, final long initialWallClockTimeMs) { - this(topology.internalTopologyBuilder, config, initialWallClockTimeMs); } - /** - * Create a new test diver instance. - * - * @param builder builder for the topology to be tested - * @param config the configuration for the topology - */ - TopologyTestDriver(final InternalTopologyBuilder builder, - final Properties config) { - this(builder, config, System.currentTimeMillis()); - - } - /** * Create a new test diver instance. * @@ -240,16 +233,16 @@ public TopologyTestDriver(final Topology topology, * @param initialWallClockTimeMs the initial value of internally mocked wall-clock time */ private TopologyTestDriver(final InternalTopologyBuilder builder, - final Properties config, - final long initialWallClockTimeMs) { + final Properties config, + final long initialWallClockTimeMs) { final StreamsConfig streamsConfig = new StreamsConfig(config); - mockTime = new MockTime(initialWallClockTimeMs); + mockWallClockTime = new MockTime(initialWallClockTimeMs); internalTopologyBuilder = builder; internalTopologyBuilder.setApplicationId(streamsConfig.getString(StreamsConfig.APPLICATION_ID_CONFIG)); processorTopology = internalTopologyBuilder.build(null); - final ProcessorTopology globalTopology = internalTopologyBuilder.buildGlobalStateTopology(); + globalTopology = internalTopologyBuilder.buildGlobalStateTopology(); final Serializer bytesSerializer = new ByteArraySerializer(); producer = new MockProducer(true, bytesSerializer, bytesSerializer) { @@ -260,7 +253,7 @@ public List partitionsFor(final String topic) { }; final MockConsumer consumer = new MockConsumer<>(OffsetResetStrategy.EARLIEST); - stateDirectory = new StateDirectory(streamsConfig, mockTime); + stateDirectory = new StateDirectory(streamsConfig, mockWallClockTime); metrics = new Metrics(); final StreamsMetricsImpl streamsMetrics = new StreamsMetricsImpl( metrics, @@ -323,6 +316,7 @@ public void onRestoreEnd(final TopicPartition topicPartition, final String store new LogContext() ); globalStateTask.initialize(); + globalProcessorContext.setRecordContext(new ProcessorRecordContext(0L, -1L, -1, ProcessorContextImpl.NONEXIST_TOPIC, new RecordHeaders())); } else { globalStateManager = null; globalStateTask = null; @@ -336,19 +330,24 @@ public void onRestoreEnd(final TopicPartition topicPartition, final String store consumer, new StoreChangelogReader( createRestoreConsumer(processorTopology.storeToChangelogTopic()), + Duration.ZERO, stateRestoreListener, new LogContext("topology-test-driver ")), streamsConfig, streamsMetrics, stateDirectory, cache, - mockTime, - producer); + mockWallClockTime, + () -> producer); task.initializeStateStores(); task.initializeTopology(); + context = (InternalProcessorContext) task.context(); + context.setRecordContext(new ProcessorRecordContext(0L, -1L, -1, ProcessorContextImpl.NONEXIST_TOPIC, new RecordHeaders())); } else { task = null; + context = null; } + eosEnabled = streamsConfig.getString(StreamsConfig.PROCESSING_GUARANTEE_CONFIG).equals(StreamsConfig.EXACTLY_ONCE); } /** @@ -356,6 +355,7 @@ public void onRestoreEnd(final TopicPartition topicPartition, final String store * * @return Map of all metrics. */ + @SuppressWarnings("WeakerAccess") public Map metrics() { return Collections.unmodifiableMap(metrics.metrics()); } @@ -390,13 +390,10 @@ public void pipeInput(final ConsumerRecord consumerRecord) { consumerRecord.headers()))); // Process the record ... - ((InternalProcessorContext) task.context()).setRecordContext( - new ProcessorRecordContext(consumerRecord.timestamp(), offset, topicPartition.partition(), topicName, consumerRecord.headers())); task.process(); task.maybePunctuateStreamTime(); task.commit(); captureOutputRecords(); - } else { final TopicPartition globalTopicPartition = globalPartitionsByTopic.get(topicName); if (globalTopicPartition == null) { @@ -445,13 +442,12 @@ private void captureOutputRecords() { // Capture all the records sent to the producer ... final List> output = producer.history(); producer.clear(); + if (eosEnabled && !producer.closed()) { + producer.initTransactions(); + producer.beginTransaction(); + } for (final ProducerRecord record : output) { - Queue> outputRecords = outputRecordsByTopic.get(record.topic()); - if (outputRecords == null) { - outputRecords = new LinkedList<>(); - outputRecordsByTopic.put(record.topic(), outputRecords); - } - outputRecords.add(record); + outputRecordsByTopic.computeIfAbsent(record.topic(), k -> new LinkedList<>()).add(record); // Forward back into the topology if the produced record is to an internal or a source topic ... final String outputTopicName = record.topic(); @@ -497,7 +493,7 @@ public void pipeInput(final List> records) { */ @SuppressWarnings("WeakerAccess") public void advanceWallClockTime(final long advanceMs) { - mockTime.sleep(advanceMs); + mockWallClockTime.sleep(advanceMs); if (task != null) { task.maybePunctuateSystemTime(); task.commit(); @@ -549,6 +545,8 @@ public ProducerRecord readOutput(final String topic, *

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

                        + * Note, that {@code StateStore} might be {@code null} if a store is added but not connected to any processor. * * @return all stores my name * @see #getStateStore(String) @@ -579,13 +577,24 @@ public Map getAllStateStores() { * @see #getWindowStore(String) * @see #getSessionStore(String) */ + @SuppressWarnings("WeakerAccess") public StateStore getStateStore(final String name) { - StateStore stateStore = task == null ? null : - ((ProcessorContextImpl) task.context()).getStateMgr().getStore(name); - if (stateStore == null && globalStateManager != null) { - stateStore = globalStateManager.getGlobalStore(name); + if (task != null) { + final StateStore stateStore = ((ProcessorContextImpl) task.context()).getStateMgr().getStore(name); + if (stateStore != null) { + return stateStore; + } } - return stateStore; + + if (globalStateManager != null) { + final StateStore stateStore = globalStateManager.getGlobalStore(name); + if (stateStore != null) { + return stateStore; + } + + } + + return null; } /** @@ -651,6 +660,7 @@ public SessionStore getSessionStore(final String name) { /** * Close the driver, its topology, and all processors. */ + @SuppressWarnings("WeakerAccess") public void close() { if (task != null) { task.close(true, false); @@ -663,9 +673,16 @@ public void close() { } } captureOutputRecords(); + if (!eosEnabled) { + producer.close(); + } stateDirectory.clean(); } + private Producer get() { + return producer; + } + static class MockTime implements Time { private final AtomicLong timeMs; private final AtomicLong highResTimeNs; diff --git a/streams/test-utils/src/main/java/org/apache/kafka/streams/processor/MockProcessorContext.java b/streams/test-utils/src/main/java/org/apache/kafka/streams/processor/MockProcessorContext.java index b14a7915dc96f..cba02573b59da 100644 --- a/streams/test-utils/src/main/java/org/apache/kafka/streams/processor/MockProcessorContext.java +++ b/streams/test-utils/src/main/java/org/apache/kafka/streams/processor/MockProcessorContext.java @@ -86,23 +86,27 @@ private CapturedPunctuator(final long intervalMs, final PunctuationType type, fi this.punctuator = punctuator; } + @SuppressWarnings({"WeakerAccess", "unused"}) public long getIntervalMs() { return intervalMs; } + @SuppressWarnings({"WeakerAccess", "unused"}) public PunctuationType getType() { return type; } + @SuppressWarnings({"WeakerAccess", "unused"}) public Punctuator getPunctuator() { return punctuator; } - @SuppressWarnings("WeakerAccess") + @SuppressWarnings({"WeakerAccess", "unused"}) public void cancel() { cancelled = true; } + @SuppressWarnings({"WeakerAccess", "unused"}) public boolean cancelled() { return cancelled; } @@ -127,6 +131,7 @@ private CapturedForward(final To to, final KeyValue keyValue) { * * @return The child name, or {@code null} if it was broadcast. */ + @SuppressWarnings({"WeakerAccess", "unused"}) public String childName() { return childName; } @@ -136,6 +141,7 @@ public String childName() { * * @return A timestamp, or {@code -1} if none was forwarded. */ + @SuppressWarnings({"WeakerAccess", "unused"}) public long timestamp() { return timestamp; } @@ -145,6 +151,7 @@ public long timestamp() { * * @return A key/value pair. Not null. */ + @SuppressWarnings({"WeakerAccess", "unused"}) public KeyValue keyValue() { return keyValue; } @@ -158,6 +165,7 @@ public KeyValue keyValue() { * and most unit tests should be able to get by with the * {@link InMemoryKeyValueStore}, so the stateDir won't matter. */ + @SuppressWarnings({"WeakerAccess", "unused"}) public MockProcessorContext() { //noinspection DoubleBraceInitialization this( @@ -179,6 +187,7 @@ public MockProcessorContext() { * * @param config a Properties object, used to configure the context and the processor. */ + @SuppressWarnings({"WeakerAccess", "unused"}) public MockProcessorContext(final Properties config) { this(config, new TaskId(0, 0), null); } @@ -190,6 +199,7 @@ public MockProcessorContext(final Properties config) { * @param taskId a {@link TaskId}, which the context makes available via {@link MockProcessorContext#taskId()}. * @param stateDir a {@link File}, which the context makes available viw {@link MockProcessorContext#stateDir()}. */ + @SuppressWarnings({"WeakerAccess", "unused"}) public MockProcessorContext(final Properties config, final TaskId taskId, final File stateDir) { final StreamsConfig streamsConfig = new StreamsConfig(config); this.taskId = taskId; @@ -252,6 +262,7 @@ public StreamsMetrics metrics() { * @param offset A record offset * @param timestamp A record timestamp */ + @SuppressWarnings({"WeakerAccess", "unused"}) public void setRecordMetadata(final String topic, final int partition, final long offset, final Headers headers, final long timestamp) { this.topic = topic; this.partition = partition; @@ -260,13 +271,13 @@ public void setRecordMetadata(final String topic, final int partition, final lon this.timestamp = timestamp; } - /** * The context exposes this metadata for use in the processor. Normally, they are set by the Kafka Streams framework, * but for the purpose of driving unit tests, you can set it directly. Setting this attribute doesn't affect the others. * * @param topic A topic name */ + @SuppressWarnings({"WeakerAccess", "unused"}) public void setTopic(final String topic) { this.topic = topic; } @@ -277,21 +288,29 @@ public void setTopic(final String topic) { * * @param partition A partition number */ + @SuppressWarnings({"WeakerAccess", "unused"}) public void setPartition(final int partition) { this.partition = partition; } - /** * The context exposes this metadata for use in the processor. Normally, they are set by the Kafka Streams framework, * but for the purpose of driving unit tests, you can set it directly. Setting this attribute doesn't affect the others. * * @param offset A record offset */ + @SuppressWarnings({"WeakerAccess", "unused"}) public void setOffset(final long offset) { this.offset = offset; } + /** + * The context exposes this metadata for use in the processor. Normally, they are set by the Kafka Streams framework, + * but for the purpose of driving unit tests, you can set it directly. Setting this attribute doesn't affect the others. + * + * @param headers Record headers + */ + @SuppressWarnings({"WeakerAccess", "unused"}) public void setHeaders(final Headers headers) { this.headers = headers; } @@ -302,6 +321,7 @@ public void setHeaders(final Headers headers) { * * @param timestamp A record timestamp */ + @SuppressWarnings({"WeakerAccess", "unused"}) public void setTimestamp(final long timestamp) { this.timestamp = timestamp; } @@ -345,7 +365,6 @@ public long timestamp() { // mocks ================================================ - @Override public void register(final StateStore store, final StateRestoreCallback stateRestoreCallbackIsIgnoredInMock) { @@ -376,6 +395,7 @@ public void cancel() { * * @return A list of captured punctuators. */ + @SuppressWarnings({"WeakerAccess", "unused"}) public List scheduledPunctuators() { final LinkedList capturedPunctuators = new LinkedList<>(); capturedPunctuators.addAll(punctuators); @@ -394,6 +414,7 @@ public void forward(final K key, final V value, final To to) { capturedForwards.add(new CapturedForward(to, new KeyValue(key, value))); } + @SuppressWarnings("deprecation") @Override public void forward(final K key, final V value, final int childIndex) { throw new UnsupportedOperationException( @@ -402,6 +423,7 @@ public void forward(final K key, final V value, final int childIndex) { ); } + @SuppressWarnings("deprecation") @Override public void forward(final K key, final V value, final String childName) { throw new UnsupportedOperationException( @@ -417,6 +439,7 @@ public void forward(final K key, final V value, final String childName) { * * @return A list of key/value pairs that were previously passed to the context. */ + @SuppressWarnings({"WeakerAccess", "unused"}) public List forwarded() { final LinkedList result = new LinkedList<>(); result.addAll(capturedForwards); @@ -431,6 +454,7 @@ public List forwarded() { * @param childName The child name to retrieve forwards for * @return A list of key/value pairs that were previously passed to the context. */ + @SuppressWarnings({"WeakerAccess", "unused"}) public List forwarded(final String childName) { final LinkedList result = new LinkedList<>(); for (final CapturedForward capture : capturedForwards) { @@ -444,6 +468,7 @@ public List forwarded(final String childName) { /** * Clear the captured forwarded data. */ + @SuppressWarnings({"WeakerAccess", "unused"}) public void resetForwards() { capturedForwards.clear(); } @@ -458,6 +483,7 @@ public void commit() { * * @return {@code true} iff {@link ProcessorContext#commit()} has been called in this context since construction or reset. */ + @SuppressWarnings("WeakerAccess") public boolean committed() { return committed; } @@ -465,6 +491,7 @@ public boolean committed() { /** * Reset the commit capture to {@code false} (whether or not it was previously {@code true}). */ + @SuppressWarnings({"WeakerAccess", "unused"}) public void resetCommit() { committed = false; } diff --git a/streams/test-utils/src/main/java/org/apache/kafka/streams/test/ConsumerRecordFactory.java b/streams/test-utils/src/main/java/org/apache/kafka/streams/test/ConsumerRecordFactory.java index 507249d0d2ee1..108dafdfdba88 100644 --- a/streams/test-utils/src/main/java/org/apache/kafka/streams/test/ConsumerRecordFactory.java +++ b/streams/test-utils/src/main/java/org/apache/kafka/streams/test/ConsumerRecordFactory.java @@ -44,7 +44,7 @@ public class ConsumerRecordFactory { private final Serializer keySerializer; private final Serializer valueSerializer; private long timeMs; - private long advanceMs; + private final long advanceMs; /** * Create a new factory for the given topic. @@ -54,6 +54,7 @@ public class ConsumerRecordFactory { * @param keySerializer the key serializer * @param valueSerializer the value serializer */ + @SuppressWarnings({"WeakerAccess", "unused"}) public ConsumerRecordFactory(final Serializer keySerializer, final Serializer valueSerializer) { this(null, keySerializer, valueSerializer, System.currentTimeMillis()); @@ -68,6 +69,7 @@ public ConsumerRecordFactory(final Serializer keySerializer, * @param keySerializer the key serializer * @param valueSerializer the value serializer */ + @SuppressWarnings({"WeakerAccess", "unused"}) public ConsumerRecordFactory(final String defaultTopicName, final Serializer keySerializer, final Serializer valueSerializer) { @@ -82,6 +84,7 @@ public ConsumerRecordFactory(final String defaultTopicName, * @param valueSerializer the value serializer * @param startTimestampMs the initial timestamp for generated records */ + @SuppressWarnings({"WeakerAccess", "unused"}) public ConsumerRecordFactory(final Serializer keySerializer, final Serializer valueSerializer, final long startTimestampMs) { @@ -97,6 +100,7 @@ public ConsumerRecordFactory(final Serializer keySerializer, * @param valueSerializer the value serializer * @param startTimestampMs the initial timestamp for generated records */ + @SuppressWarnings({"WeakerAccess", "unused"}) public ConsumerRecordFactory(final String defaultTopicName, final Serializer keySerializer, final Serializer valueSerializer, @@ -112,6 +116,7 @@ public ConsumerRecordFactory(final String defaultTopicName, * @param startTimestampMs the initial timestamp for generated records * @param autoAdvanceMs the time increment pre generated record */ + @SuppressWarnings({"WeakerAccess", "unused"}) public ConsumerRecordFactory(final Serializer keySerializer, final Serializer valueSerializer, final long startTimestampMs, @@ -128,6 +133,7 @@ public ConsumerRecordFactory(final Serializer keySerializer, * @param startTimestampMs the initial timestamp for generated records * @param autoAdvanceMs the time increment pre generated record */ + @SuppressWarnings({"WeakerAccess", "unused"}) public ConsumerRecordFactory(final String defaultTopicName, final Serializer keySerializer, final Serializer valueSerializer, @@ -147,6 +153,7 @@ public ConsumerRecordFactory(final String defaultTopicName, * * @param advanceMs the amount of time to advance */ + @SuppressWarnings({"WeakerAccess", "unused"}) public void advanceTimeMs(final long advanceMs) { if (advanceMs < 0) { throw new IllegalArgumentException("advanceMs must be positive"); @@ -165,6 +172,7 @@ public void advanceTimeMs(final long advanceMs) { * @param timestampMs the record timestamp * @return the generated {@link ConsumerRecord} */ + @SuppressWarnings({"WeakerAccess", "unused"}) public ConsumerRecord create(final String topicName, final K key, final V value, @@ -198,6 +206,7 @@ public ConsumerRecord create(final String topicName, * @param timestampMs the record timestamp * @return the generated {@link ConsumerRecord} */ + @SuppressWarnings({"WeakerAccess", "unused"}) public ConsumerRecord create(final String topicName, final K key, final V value, @@ -214,6 +223,7 @@ public ConsumerRecord create(final String topicName, * @param timestampMs the record timestamp * @return the generated {@link ConsumerRecord} */ + @SuppressWarnings({"WeakerAccess", "unused"}) public ConsumerRecord create(final K key, final V value, final long timestampMs) { @@ -230,6 +240,7 @@ public ConsumerRecord create(final K key, * @param timestampMs the record timestamp * @return the generated {@link ConsumerRecord} */ + @SuppressWarnings({"WeakerAccess", "unused"}) public ConsumerRecord create(final K key, final V value, final Headers headers, @@ -250,6 +261,7 @@ public ConsumerRecord create(final K key, * @param value the record value * @return the generated {@link ConsumerRecord} */ + @SuppressWarnings({"WeakerAccess", "unused"}) public ConsumerRecord create(final String topicName, final K key, final V value) { @@ -268,6 +280,7 @@ public ConsumerRecord create(final String topicName, * @param headers the record headers * @return the generated {@link ConsumerRecord} */ + @SuppressWarnings({"WeakerAccess", "unused"}) public ConsumerRecord create(final String topicName, final K key, final V value, @@ -285,6 +298,7 @@ public ConsumerRecord create(final String topicName, * @param value the record value * @return the generated {@link ConsumerRecord} */ + @SuppressWarnings({"WeakerAccess", "unused"}) public ConsumerRecord create(final K key, final V value) { return create(key, value, new RecordHeaders()); @@ -299,6 +313,7 @@ public ConsumerRecord create(final K key, * @param headers the record headers * @return the generated {@link ConsumerRecord} */ + @SuppressWarnings({"WeakerAccess", "unused"}) public ConsumerRecord create(final K key, final V value, final Headers headers) { @@ -318,6 +333,7 @@ public ConsumerRecord create(final K key, * @param timestampMs the record timestamp * @return the generated {@link ConsumerRecord} */ + @SuppressWarnings({"WeakerAccess", "unused"}) public ConsumerRecord create(final String topicName, final V value, final long timestampMs) { @@ -334,6 +350,7 @@ public ConsumerRecord create(final String topicName, * @param timestampMs the record timestamp * @return the generated {@link ConsumerRecord} */ + @SuppressWarnings({"WeakerAccess", "unused"}) public ConsumerRecord create(final String topicName, final V value, final Headers headers, @@ -349,6 +366,7 @@ public ConsumerRecord create(final String topicName, * @param timestampMs the record timestamp * @return the generated {@link ConsumerRecord} */ + @SuppressWarnings({"WeakerAccess", "unused"}) public ConsumerRecord create(final V value, final long timestampMs) { return create(value, new RecordHeaders(), timestampMs); @@ -363,6 +381,7 @@ public ConsumerRecord create(final V value, * @param timestampMs the record timestamp * @return the generated {@link ConsumerRecord} */ + @SuppressWarnings({"WeakerAccess", "unused"}) public ConsumerRecord create(final V value, final Headers headers, final long timestampMs) { @@ -382,6 +401,7 @@ public ConsumerRecord create(final V value, * @param headers the record headers * @return the generated {@link ConsumerRecord} */ + @SuppressWarnings({"WeakerAccess", "unused"}) public ConsumerRecord create(final String topicName, final V value, final Headers headers) { @@ -396,6 +416,7 @@ public ConsumerRecord create(final String topicName, * @param value the record value * @return the generated {@link ConsumerRecord} */ + @SuppressWarnings({"WeakerAccess", "unused"}) public ConsumerRecord create(final String topicName, final V value) { return create(topicName, null, value, new RecordHeaders()); @@ -408,6 +429,7 @@ public ConsumerRecord create(final String topicName, * @param value the record value * @return the generated {@link ConsumerRecord} */ + @SuppressWarnings({"WeakerAccess", "unused"}) public ConsumerRecord create(final V value) { return create(value, new RecordHeaders()); } @@ -420,6 +442,7 @@ public ConsumerRecord create(final V value) { * @param headers the record headers * @return the generated {@link ConsumerRecord} */ + @SuppressWarnings({"WeakerAccess", "unused"}) public ConsumerRecord create(final V value, final Headers headers) { if (topicName == null) { @@ -437,6 +460,7 @@ public ConsumerRecord create(final V value, * @param keyValues the record keys and values * @return the generated {@link ConsumerRecord consumer records} */ + @SuppressWarnings({"WeakerAccess", "unused"}) public List> create(final String topicName, final List> keyValues) { final List> records = new ArrayList<>(keyValues.size()); @@ -455,6 +479,7 @@ public List> create(final String topicName, * @param keyValues the record keys and values * @return the generated {@link ConsumerRecord consumer records} */ + @SuppressWarnings({"WeakerAccess", "unused"}) public List> create(final List> keyValues) { if (topicName == null) { throw new IllegalStateException("ConsumerRecordFactory was created without defaultTopicName. " + @@ -474,6 +499,7 @@ public List> create(final List> ke * @param advanceMs the time difference between two consecutive generated records * @return the generated {@link ConsumerRecord consumer records} */ + @SuppressWarnings({"WeakerAccess", "unused"}) public List> create(final String topicName, final List> keyValues, final long startTimestamp, @@ -502,6 +528,7 @@ public List> create(final String topicName, * @param advanceMs the time difference between two consecutive generated records * @return the generated {@link ConsumerRecord consumer records} */ + @SuppressWarnings({"WeakerAccess", "unused"}) public List> create(final List> keyValues, final long startTimestamp, final long advanceMs) { @@ -523,6 +550,7 @@ public List> create(final List> ke * @param startTimestamp the timestamp for the first generated record * @return the generated {@link ConsumerRecord consumer records} */ + @SuppressWarnings({"WeakerAccess", "unused"}) public List> create(final String topicName, final List> keyValues, final long startTimestamp) { @@ -538,6 +566,7 @@ public List> create(final String topicName, * @param startTimestamp the timestamp for the first generated record * @return the generated {@link ConsumerRecord consumer records} */ + @SuppressWarnings({"WeakerAccess", "unused"}) public List> create(final List> keyValues, final long startTimestamp) { if (topicName == null) { diff --git a/streams/test-utils/src/main/java/org/apache/kafka/streams/test/OutputVerifier.java b/streams/test-utils/src/main/java/org/apache/kafka/streams/test/OutputVerifier.java index aedb910e28c1e..f78e926e4316c 100644 --- a/streams/test-utils/src/main/java/org/apache/kafka/streams/test/OutputVerifier.java +++ b/streams/test-utils/src/main/java/org/apache/kafka/streams/test/OutputVerifier.java @@ -39,6 +39,7 @@ public class OutputVerifier { * @param the value type * @throws AssertionError if {@code ProducerRecord}'s value is not equal to {@code expectedValue} */ + @SuppressWarnings({"WeakerAccess", "unused"}) public static void compareValue(final ProducerRecord record, final V expectedValue) throws AssertionError { Objects.requireNonNull(record); @@ -65,6 +66,7 @@ public static void compareValue(final ProducerRecord record, * @param the value type * @throws AssertionError if {@code ProducerRecord}'s value is not equal to {@code expectedRecord}'s value */ + @SuppressWarnings({"WeakerAccess", "unused"}) public static void compareValue(final ProducerRecord record, final ProducerRecord expectedRecord) throws AssertionError { Objects.requireNonNull(expectedRecord); @@ -82,6 +84,7 @@ public static void compareValue(final ProducerRecord record, * @param the value type * @throws AssertionError if {@code ProducerRecord}'s key or value is not equal to {@code expectedKey} or {@code expectedValue} */ + @SuppressWarnings({"WeakerAccess", "unused"}) public static void compareKeyValue(final ProducerRecord record, final K expectedKey, final V expectedValue) throws AssertionError { @@ -119,6 +122,7 @@ public static void compareKeyValue(final ProducerRecord record, * @param the value type * @throws AssertionError if {@code ProducerRecord}'s key or value is not equal to {@code expectedRecord}'s key or value */ + @SuppressWarnings({"WeakerAccess", "unused"}) public static void compareKeyValue(final ProducerRecord record, final ProducerRecord expectedRecord) throws AssertionError { Objects.requireNonNull(expectedRecord); @@ -136,6 +140,7 @@ public static void compareKeyValue(final ProducerRecord record, * @param the value type * @throws AssertionError if {@code ProducerRecord}'s value or timestamp is not equal to {@code expectedValue} or {@code expectedTimestamp} */ + @SuppressWarnings({"WeakerAccess", "unused"}) public static void compareValueTimestamp(final ProducerRecord record, final V expectedValue, final long expectedTimestamp) throws AssertionError { @@ -169,6 +174,7 @@ public static void compareValueTimestamp(final ProducerRecord recor * @param the value type * @throws AssertionError if {@code ProducerRecord}'s value or timestamp is not equal to {@code expectedRecord}'s value or timestamp */ + @SuppressWarnings({"WeakerAccess", "unused"}) public static void compareValueTimestamp(final ProducerRecord record, final ProducerRecord expectedRecord) throws AssertionError { Objects.requireNonNull(expectedRecord); @@ -189,6 +195,7 @@ public static void compareValueTimestamp(final ProducerRecord recor * @throws AssertionError if {@code ProducerRecord}'s key, value, timestamp is not equal to {@code expectedKey}, * {@code expectedValue}, or {@code expectedTimestamps} */ + @SuppressWarnings({"WeakerAccess", "unused"}) public static void compareKeyValueTimestamp(final ProducerRecord record, final K expectedKey, final V expectedValue, @@ -233,6 +240,7 @@ public static void compareKeyValueTimestamp(final ProducerRecord re * @throws AssertionError if {@code ProducerRecord}'s key, value, or timestamp is not equal to * {@code expectedRecord}'s key, value, or timestamp */ + @SuppressWarnings({"WeakerAccess", "unused"}) public static void compareKeyValueTimestamp(final ProducerRecord record, final ProducerRecord expectedRecord) throws AssertionError { Objects.requireNonNull(expectedRecord); @@ -250,6 +258,7 @@ public static void compareKeyValueTimestamp(final ProducerRecord re * @param the value type * @throws AssertionError if {@code ProducerRecord}'s value or headers is not equal to {@code expectedValue} or {@code expectedHeaders} */ + @SuppressWarnings({"WeakerAccess", "unused"}) public static void compareValueHeaders(final ProducerRecord record, final V expectedValue, final Headers expectedHeaders) throws AssertionError { @@ -287,6 +296,7 @@ public static void compareValueHeaders(final ProducerRecord record, * @param the value type * @throws AssertionError if {@code ProducerRecord}'s value or headers is not equal to {@code expectedRecord}'s value or headers */ + @SuppressWarnings({"WeakerAccess", "unused"}) public static void compareValueHeaders(final ProducerRecord record, final ProducerRecord expectedRecord) throws AssertionError { Objects.requireNonNull(expectedRecord); @@ -307,6 +317,7 @@ public static void compareValueHeaders(final ProducerRecord record, * @throws AssertionError if {@code ProducerRecord}'s key, value, headers is not equal to {@code expectedKey}, * {@code expectedValue}, or {@code expectedHeaders} */ + @SuppressWarnings({"WeakerAccess", "unused"}) public static void compareKeyValueHeaders(final ProducerRecord record, final K expectedKey, final V expectedValue, @@ -355,6 +366,7 @@ public static void compareKeyValueHeaders(final ProducerRecord reco * @throws AssertionError if {@code ProducerRecord}'s key, value, or headers is not equal to * {@code expectedRecord}'s key, value, or headers */ + @SuppressWarnings({"WeakerAccess", "unused"}) public static void compareKeyValueHeaders(final ProducerRecord record, final ProducerRecord expectedRecord) throws AssertionError { Objects.requireNonNull(expectedRecord); @@ -376,6 +388,7 @@ public static void compareKeyValueHeaders(final ProducerRecord reco * @throws AssertionError if {@code ProducerRecord}'s key, value, headers is not equal to {@code expectedKey}, * {@code expectedValue}, or {@code expectedHeaders} */ + @SuppressWarnings({"WeakerAccess", "unused"}) public static void compareKeyValueHeadersTimestamp(final ProducerRecord record, final K expectedKey, final V expectedValue, @@ -432,6 +445,7 @@ public static void compareKeyValueHeadersTimestamp(final ProducerRecord void compareKeyValueHeadersTimestamp(final ProducerRecord record, final ProducerRecord expectedRecord) throws AssertionError { Objects.requireNonNull(expectedRecord); diff --git a/streams/test-utils/src/test/java/org/apache/kafka/streams/MockProcessorContextTest.java b/streams/test-utils/src/test/java/org/apache/kafka/streams/MockProcessorContextTest.java index 64d5b12dc0b2e..878aa35748340 100644 --- a/streams/test-utils/src/test/java/org/apache/kafka/streams/MockProcessorContextTest.java +++ b/streams/test-utils/src/test/java/org/apache/kafka/streams/MockProcessorContextTest.java @@ -156,9 +156,9 @@ public void process(final String key, final Long value) { @Test public void shouldThrowIfForwardedWithDeprecatedChildIndex() { final AbstractProcessor processor = new AbstractProcessor() { + @SuppressWarnings("deprecation") @Override public void process(final String key, final Long value) { - //noinspection deprecation context().forward(key, value, 0); } }; @@ -178,9 +178,9 @@ public void process(final String key, final Long value) { @Test public void shouldThrowIfForwardedWithDeprecatedChildName() { final AbstractProcessor processor = new AbstractProcessor() { + @SuppressWarnings("deprecation") @Override public void process(final String key, final Long value) { - //noinspection deprecation context().forward(key, value, "child1"); } }; @@ -347,12 +347,7 @@ public void init(final ProcessorContext context) { context.schedule( 1000L, PunctuationType.WALL_CLOCK_TIME, - new Punctuator() { - @Override - public void punctuate(final long timestamp) { - context.commit(); - } - } + timestamp -> context.commit() ); } diff --git a/streams/test-utils/src/test/java/org/apache/kafka/streams/TopologyTestDriverTest.java b/streams/test-utils/src/test/java/org/apache/kafka/streams/TopologyTestDriverTest.java index 2d446d1de2ccd..8827d4254c2c1 100644 --- a/streams/test-utils/src/test/java/org/apache/kafka/streams/TopologyTestDriverTest.java +++ b/streams/test-utils/src/test/java/org/apache/kafka/streams/TopologyTestDriverTest.java @@ -31,14 +31,15 @@ import org.apache.kafka.common.serialization.StringSerializer; import org.apache.kafka.common.utils.Bytes; import org.apache.kafka.common.utils.SystemTime; +import org.apache.kafka.streams.errors.TopologyException; import org.apache.kafka.streams.kstream.Consumed; import org.apache.kafka.streams.kstream.Materialized; -import org.apache.kafka.streams.errors.TopologyException; 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.PunctuationType; import org.apache.kafka.streams.processor.Punctuator; +import org.apache.kafka.streams.processor.StateStore; import org.apache.kafka.streams.state.KeyValueIterator; import org.apache.kafka.streams.state.KeyValueStore; import org.apache.kafka.streams.state.Stores; @@ -47,10 +48,12 @@ import org.apache.kafka.streams.test.OutputVerifier; import org.apache.kafka.test.TestUtils; import org.junit.After; -import org.junit.Assert; import org.junit.Test; +import org.junit.runner.RunWith; +import org.junit.runners.Parameterized; import java.util.ArrayList; +import java.util.Arrays; import java.util.Collection; import java.util.Collections; import java.util.HashSet; @@ -63,11 +66,14 @@ import java.util.regex.Pattern; import static org.hamcrest.CoreMatchers.equalTo; +import static org.hamcrest.MatcherAssert.assertThat; import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertThat; +import static org.junit.Assert.assertNotNull; +import static org.junit.Assert.assertNull; import static org.junit.Assert.assertTrue; import static org.junit.Assert.fail; +@RunWith(value = Parameterized.class) public class TopologyTestDriverTest { private final static String SOURCE_TOPIC_1 = "source-topic-1"; private final static String SOURCE_TOPIC_2 = "source-topic-2"; @@ -100,26 +106,41 @@ public class TopologyTestDriverTest { }; private KeyValueStore store; - private StringDeserializer stringDeserializer = new StringDeserializer(); - private LongDeserializer longDeserializer = new LongDeserializer(); - private ConsumerRecordFactory recordFactory = new ConsumerRecordFactory<>( + private final StringDeserializer stringDeserializer = new StringDeserializer(); + private final LongDeserializer longDeserializer = new LongDeserializer(); + private final ConsumerRecordFactory recordFactory = new ConsumerRecordFactory<>( new StringSerializer(), new LongSerializer()); + @Parameterized.Parameters(name = "Eos enabled = {0}") + public static Collection data() { + final List values = new ArrayList<>(); + for (final boolean eosEnabled : Arrays.asList(true, false)) { + values.add(new Object[] {eosEnabled}); + } + return values; + } + + public TopologyTestDriverTest(final boolean eosEnabled) { + if (eosEnabled) { + config.put(StreamsConfig.PROCESSING_GUARANTEE_CONFIG, StreamsConfig.EXACTLY_ONCE); + } + } private final static class Record { - private Object key; - private Object value; - private long timestamp; - private long offset; - private String topic; - private Headers headers; - - Record(final ConsumerRecord consumerRecord) { + private final Object key; + private final Object value; + private final long timestamp; + private final long offset; + private final String topic; + private final Headers headers; + + Record(final ConsumerRecord consumerRecord, + final long newOffset) { key = consumerRecord.key(); value = consumerRecord.value(); timestamp = consumerRecord.timestamp(); - offset = consumerRecord.offset(); + offset = newOffset; topic = consumerRecord.topic(); headers = consumerRecord.headers(); } @@ -184,7 +205,7 @@ private final class MockPunctuator implements Punctuator { private final List punctuatedAt = new LinkedList<>(); @Override - public void punctuate(long timestamp) { + public void punctuate(final long timestamp) { punctuatedAt.add(timestamp); } } @@ -202,7 +223,7 @@ private final class MockProcessor implements Processor { } @Override - public void init(ProcessorContext context) { + public void init(final ProcessorContext context) { initialized = true; this.context = context; for (final Punctuation punctuation : punctuations) { @@ -211,7 +232,7 @@ public void init(ProcessorContext context) { } @Override - public void process(Object key, Object value) { + public void process(final Object key, final Object value) { processedRecords.add(new Record(key, value, context.headers(), context.timestamp(), context.offset(), context.topic())); context.forward(key, value); } @@ -228,7 +249,7 @@ private final class MockProcessorSupplier implements ProcessorSupplier { private final Collection punctuations; private MockProcessorSupplier() { - this(Collections.emptySet()); + this(Collections.emptySet()); } private MockProcessorSupplier(final Collection punctuations) { @@ -350,6 +371,8 @@ public void shouldCloseProcessor() { testDriver.close(); assertTrue(mockProcessors.get(0).closed); + // As testDriver is already closed, bypassing @After tearDown testDriver.close(). + testDriver = null; } @Test @@ -391,8 +414,7 @@ public void shouldSetRecordMetadata() { assertEquals(1, processedRecords.size()); final Record record = processedRecords.get(0); - final Record expectedResult = new Record(consumerRecord1); - expectedResult.offset = 0L; + final Record expectedResult = new Record(consumerRecord1, 0L); assertThat(record, equalTo(expectedResult)); } @@ -410,8 +432,7 @@ public void shouldSendRecordViaCorrectSourceTopic() { assertEquals(0, processedRecords2.size()); Record record = processedRecords1.get(0); - Record expectedResult = new Record(consumerRecord1); - expectedResult.offset = 0L; + Record expectedResult = new Record(consumerRecord1, 0L); assertThat(record, equalTo(expectedResult)); testDriver.pipeInput(consumerRecord2); @@ -420,8 +441,7 @@ public void shouldSendRecordViaCorrectSourceTopic() { assertEquals(1, processedRecords2.size()); record = processedRecords2.get(0); - expectedResult = new Record(consumerRecord2); - expectedResult.offset = 0L; + expectedResult = new Record(consumerRecord2, 0L); assertThat(record, equalTo(expectedResult)); } @@ -439,7 +459,7 @@ public void shouldUseSourceSpecificDeserializers() { topology.addSink( "sink", SINK_TOPIC_1, - new Serializer() { + new Serializer() { @Override public byte[] serialize(final String topic, final Object data) { if (data instanceof Long) { @@ -452,7 +472,7 @@ public void close() {} @Override public void configure(final Map configs, final boolean isKey) {} }, - new Serializer() { + new Serializer() { @Override public byte[] serialize(final String topic, final Object data) { if (data instanceof String) { @@ -560,13 +580,11 @@ public void shouldProcessConsumerRecordList() { assertEquals(1, processedRecords2.size()); Record record = processedRecords1.get(0); - Record expectedResult = new Record(consumerRecord1); - expectedResult.offset = 0L; + Record expectedResult = new Record(consumerRecord1, 0L); assertThat(record, equalTo(expectedResult)); record = processedRecords2.get(0); - expectedResult = new Record(consumerRecord2); - expectedResult.offset = 0L; + expectedResult = new Record(consumerRecord2, 0L); assertThat(record, equalTo(expectedResult)); } @@ -592,8 +610,8 @@ public void shouldPopulateGlobalStore() { testDriver = new TopologyTestDriver(setupGlobalStoreTopology(SOURCE_TOPIC_1), config); final KeyValueStore globalStore = testDriver.getKeyValueStore(SOURCE_TOPIC_1 + "-globalStore"); - Assert.assertNotNull(globalStore); - Assert.assertNotNull(testDriver.getAllStateStores().get(SOURCE_TOPIC_1 + "-globalStore")); + assertNotNull(globalStore); + assertNotNull(testDriver.getAllStateStores().get(SOURCE_TOPIC_1 + "-globalStore")); testDriver.pipeInput(consumerRecord1); @@ -601,8 +619,7 @@ public void shouldPopulateGlobalStore() { assertEquals(1, processedRecords.size()); final Record record = processedRecords.get(0); - final Record expectedResult = new Record(consumerRecord1); - expectedResult.offset = 0L; + final Record expectedResult = new Record(consumerRecord1, 0L); assertThat(record, equalTo(expectedResult)); } @@ -687,13 +704,14 @@ public void shouldPunctuateOnWallClockTime() { @Test public void shouldReturnAllStores() { final Topology topology = setupSourceSinkTopology(); + topology.addProcessor("processor", () -> null, "source"); topology.addStateStore( new KeyValueStoreBuilder<>( Stores.inMemoryKeyValueStore("store"), Serdes.ByteArray(), Serdes.ByteArray(), - new SystemTime()) - .withLoggingDisabled()); + new SystemTime()), + "processor"); topology.addGlobalStore( new KeyValueStoreBuilder<>( Stores.inMemoryKeyValueStore("globalStore"), @@ -705,12 +723,41 @@ public void shouldReturnAllStores() { Serdes.ByteArray().deserializer(), "globalTopicName", "globalProcessorName", - new ProcessorSupplier() { - @Override - public Processor get() { - return null; - } - }); + () -> null); + + testDriver = new TopologyTestDriver(topology, config); + + final Set expectedStoreNames = new HashSet<>(); + expectedStoreNames.add("store"); + expectedStoreNames.add("globalStore"); + final Map allStores = testDriver.getAllStateStores(); + assertThat(allStores.keySet(), equalTo(expectedStoreNames)); + for (final StateStore store : allStores.values()) { + assertNotNull(store); + } + } + + @Test + public void shouldReturnAllStoresNames() { + final Topology topology = setupSourceSinkTopology(); + topology.addStateStore( + new KeyValueStoreBuilder<>( + Stores.inMemoryKeyValueStore("store"), + Serdes.ByteArray(), + Serdes.ByteArray(), + new SystemTime())); + topology.addGlobalStore( + new KeyValueStoreBuilder<>( + Stores.inMemoryKeyValueStore("globalStore"), + Serdes.ByteArray(), + Serdes.ByteArray(), + new SystemTime()).withLoggingDisabled(), + "sourceProcessorName", + Serdes.ByteArray().deserializer(), + Serdes.ByteArray().deserializer(), + "globalTopicName", + "globalProcessorName", + () -> null); testDriver = new TopologyTestDriver(topology, config); @@ -721,13 +768,13 @@ public Processor get() { } private void setup() { - Topology topology = new Topology(); + final Topology topology = new Topology(); topology.addSource("sourceProcessor", "input-topic"); topology.addProcessor("aggregator", new CustomMaxAggregatorSupplier(), "sourceProcessor"); topology.addStateStore(Stores.keyValueStoreBuilder( Stores.inMemoryKeyValueStore("aggStore"), Serdes.String(), - Serdes.Long()).withLoggingDisabled(), // need to disable logging to allow store pre-populating + Serdes.Long()), "aggregator"); topology.addSink("sinkProcessor", "result-topic", "aggregator"); @@ -744,35 +791,35 @@ public void shouldFlushStoreForFirstInput() { setup(); testDriver.pipeInput(recordFactory.create("input-topic", "a", 1L, 9999L)); OutputVerifier.compareKeyValue(testDriver.readOutput("result-topic", stringDeserializer, longDeserializer), "a", 21L); - Assert.assertNull(testDriver.readOutput("result-topic", stringDeserializer, longDeserializer)); + assertNull(testDriver.readOutput("result-topic", stringDeserializer, longDeserializer)); } @Test public void shouldNotUpdateStoreForSmallerValue() { setup(); testDriver.pipeInput(recordFactory.create("input-topic", "a", 1L, 9999L)); - Assert.assertThat(store.get("a"), equalTo(21L)); + assertThat(store.get("a"), equalTo(21L)); OutputVerifier.compareKeyValue(testDriver.readOutput("result-topic", stringDeserializer, longDeserializer), "a", 21L); - Assert.assertNull(testDriver.readOutput("result-topic", stringDeserializer, longDeserializer)); + assertNull(testDriver.readOutput("result-topic", stringDeserializer, longDeserializer)); } @Test public void shouldNotUpdateStoreForLargerValue() { setup(); testDriver.pipeInput(recordFactory.create("input-topic", "a", 42L, 9999L)); - Assert.assertThat(store.get("a"), equalTo(42L)); + assertThat(store.get("a"), equalTo(42L)); OutputVerifier.compareKeyValue(testDriver.readOutput("result-topic", stringDeserializer, longDeserializer), "a", 42L); - Assert.assertNull(testDriver.readOutput("result-topic", stringDeserializer, longDeserializer)); + assertNull(testDriver.readOutput("result-topic", stringDeserializer, longDeserializer)); } @Test public void shouldUpdateStoreForNewKey() { setup(); testDriver.pipeInput(recordFactory.create("input-topic", "b", 21L, 9999L)); - Assert.assertThat(store.get("b"), equalTo(21L)); + assertThat(store.get("b"), equalTo(21L)); OutputVerifier.compareKeyValue(testDriver.readOutput("result-topic", stringDeserializer, longDeserializer), "a", 21L); OutputVerifier.compareKeyValue(testDriver.readOutput("result-topic", stringDeserializer, longDeserializer), "b", 21L); - Assert.assertNull(testDriver.readOutput("result-topic", stringDeserializer, longDeserializer)); + assertNull(testDriver.readOutput("result-topic", stringDeserializer, longDeserializer)); } @Test @@ -782,11 +829,11 @@ public void shouldPunctuateIfEvenTimeAdvances() { OutputVerifier.compareKeyValue(testDriver.readOutput("result-topic", stringDeserializer, longDeserializer), "a", 21L); testDriver.pipeInput(recordFactory.create("input-topic", "a", 1L, 9999L)); - Assert.assertNull(testDriver.readOutput("result-topic", stringDeserializer, longDeserializer)); + assertNull(testDriver.readOutput("result-topic", stringDeserializer, longDeserializer)); testDriver.pipeInput(recordFactory.create("input-topic", "a", 1L, 10000L)); OutputVerifier.compareKeyValue(testDriver.readOutput("result-topic", stringDeserializer, longDeserializer), "a", 21L); - Assert.assertNull(testDriver.readOutput("result-topic", stringDeserializer, longDeserializer)); + assertNull(testDriver.readOutput("result-topic", stringDeserializer, longDeserializer)); } @Test @@ -794,7 +841,7 @@ public void shouldPunctuateIfWallClockTimeAdvances() { setup(); testDriver.advanceWallClockTime(60000); OutputVerifier.compareKeyValue(testDriver.readOutput("result-topic", stringDeserializer, longDeserializer), "a", 21L); - Assert.assertNull(testDriver.readOutput("result-topic", stringDeserializer, longDeserializer)); + assertNull(testDriver.readOutput("result-topic", stringDeserializer, longDeserializer)); } private class CustomMaxAggregatorSupplier implements ProcessorSupplier { @@ -812,18 +859,8 @@ private class CustomMaxAggregator implements Processor { @Override public void init(final ProcessorContext context) { this.context = context; - context.schedule(60000, PunctuationType.WALL_CLOCK_TIME, new Punctuator() { - @Override - public void punctuate(final long timestamp) { - flushStore(); - } - }); - context.schedule(10000, PunctuationType.STREAM_TIME, new Punctuator() { - @Override - public void punctuate(final long timestamp) { - flushStore(); - } - }); + context.schedule(60000, PunctuationType.WALL_CLOCK_TIME, timestamp -> flushStore()); + context.schedule(10000, PunctuationType.STREAM_TIME, timestamp -> flushStore()); store = (KeyValueStore) context.getStateStore("aggStore"); } @@ -836,10 +873,11 @@ public void process(final String key, final Long value) { } private void flushStore() { - final KeyValueIterator it = store.all(); - while (it.hasNext()) { - final KeyValue next = it.next(); - context.forward(next.key, next.value); + try (final KeyValueIterator it = store.all()) { + while (it.hasNext()) { + final KeyValue next = it.next(); + context.forward(next.key, next.value); + } } } @@ -847,6 +885,23 @@ private void flushStore() { public void close() {} } + @Test + public void shouldAllowPrePopulatingStatesStoresWithCachingEnabled() { + final Topology topology = new Topology(); + topology.addSource("sourceProcessor", "input-topic"); + topology.addProcessor("aggregator", new CustomMaxAggregatorSupplier(), "sourceProcessor"); + topology.addStateStore(Stores.keyValueStoreBuilder( + Stores.inMemoryKeyValueStore("aggStore"), + Serdes.String(), + Serdes.Long()).withCachingEnabled(), // intentionally turn on caching to achieve better test coverage + "aggregator"); + + testDriver = new TopologyTestDriver(topology, config); + + store = testDriver.getKeyValueStore("aggStore"); + store.put("a", 21L); + } + @Test public void shouldCleanUpPersistentStateStoresOnClose() { final Topology topology = new Topology(); @@ -886,21 +941,20 @@ public void close() {} config.put(StreamsConfig.DEFAULT_KEY_SERDE_CLASS_CONFIG, Serdes.String().getClass().getName()); config.put(StreamsConfig.DEFAULT_VALUE_SERDE_CLASS_CONFIG, Serdes.Long().getClass().getName()); - { - final TopologyTestDriver testDriver = new TopologyTestDriver(topology, config); - Assert.assertNull(testDriver.getKeyValueStore("storeProcessorStore").get("a")); + try (final TopologyTestDriver testDriver = new TopologyTestDriver(topology, config)) { + assertNull(testDriver.getKeyValueStore("storeProcessorStore").get("a")); testDriver.pipeInput(recordFactory.create("input-topic", "a", 1L)); - Assert.assertEquals(1L, testDriver.getKeyValueStore("storeProcessorStore").get("a")); - testDriver.close(); + assertEquals(1L, testDriver.getKeyValueStore("storeProcessorStore").get("a")); } - { - final TopologyTestDriver testDriver = new TopologyTestDriver(topology, config); - Assert.assertNull( + + try (final TopologyTestDriver testDriver = new TopologyTestDriver(topology, config)) { + assertNull( "Closing the prior test driver should have cleaned up this store and value.", testDriver.getKeyValueStore("storeProcessorStore").get("a") ); } + } @Test @@ -908,15 +962,15 @@ public void shouldFeedStoreFromGlobalKTable() { final StreamsBuilder builder = new StreamsBuilder(); builder.globalTable("topic", Consumed.with(Serdes.String(), Serdes.String()), - Materialized.>as("globalStore")); + Materialized.as("globalStore")); try (final TopologyTestDriver testDriver = new TopologyTestDriver(builder.build(), config)) { final KeyValueStore globalStore = testDriver.getKeyValueStore("globalStore"); - Assert.assertNotNull(globalStore); - Assert.assertNotNull(testDriver.getAllStateStores().get("globalStore")); + assertNotNull(globalStore); + assertNotNull(testDriver.getAllStateStores().get("globalStore")); final ConsumerRecordFactory recordFactory = new ConsumerRecordFactory<>(new StringSerializer(), new StringSerializer()); testDriver.pipeInput(recordFactory.create("topic", "k1", "value1")); // we expect to have both in the global store, the one from pipeInput and the one from the producer - Assert.assertEquals("value1", globalStore.get("k1")); + assertEquals("value1", globalStore.get("k1")); } } @@ -956,8 +1010,7 @@ public void shouldProcessFromSourcesThatMatchMultiplePattern() { assertEquals(0, processedRecords2.size()); final Record record1 = processedRecords1.get(0); - final Record expectedResult1 = new Record(consumerRecord1); - expectedResult1.offset = 0L; + final Record expectedResult1 = new Record(consumerRecord1, 0L); assertThat(record1, equalTo(expectedResult1)); testDriver.pipeInput(consumerRecord2); @@ -966,8 +1019,7 @@ public void shouldProcessFromSourcesThatMatchMultiplePattern() { assertEquals(1, processedRecords2.size()); final Record record2 = processedRecords2.get(0); - final Record expectedResult2 = new Record(consumerRecord2); - expectedResult2.offset = 0L; + final Record expectedResult2 = new Record(consumerRecord2, 0L); assertThat(record2, equalTo(expectedResult2)); } @@ -1004,7 +1056,7 @@ public void shouldThrowPatternNotValidForTopicNameException() { try { testDriver.pipeInput(consumerRecord1); } catch (final TopologyException exception) { - String str = + final String str = String.format( "Invalid topology: Topology add source of type String for topic: %s cannot contain regex pattern for " + "input record topic: %s and hence cannot process the message.", diff --git a/tests/README.md b/tests/README.md index f0ffdf521f36f..f42b28ac22d75 100644 --- a/tests/README.md +++ b/tests/README.md @@ -461,6 +461,7 @@ the test driver machine. ec2_instance_type = "..." # Pick something appropriate for your # test. Note that the default m3.medium has # a small disk. + ec2_spot_max_price = "0.123" # On-demand price for instance type enable_hostmanager = false num_zookeepers = 0 num_kafka = 0 diff --git a/tests/docker/Dockerfile b/tests/docker/Dockerfile index f1239a919e5b7..3d577b6242628 100644 --- a/tests/docker/Dockerfile +++ b/tests/docker/Dockerfile @@ -32,7 +32,7 @@ LABEL ducker.creator=$ducker_creator # Update Linux and install necessary utilities. RUN apt update && apt install -y sudo netcat iptables rsync unzip wget curl jq coreutils openssh-server net-tools vim python-pip python-dev libffi-dev libssl-dev cmake pkg-config libfuse-dev && apt-get -y clean -RUN pip install -U pip==9.0.3 setuptools && pip install --upgrade cffi virtualenv pyasn1 boto3 pycrypto pywinrm ipaddress enum34 && pip install --upgrade ducktape==0.7.1 +RUN pip install -U pip==9.0.3 setuptools && pip install --upgrade cffi virtualenv pyasn1 boto3 pycrypto pywinrm ipaddress enum34 && pip install --upgrade ducktape==0.7.5 # Set up ssh COPY ./ssh-config /root/.ssh/config @@ -43,32 +43,20 @@ RUN ssh-keygen -q -t rsa -N '' -f /root/.ssh/id_rsa && cp -f /root/.ssh/id_rsa.p ARG KAFKA_MIRROR="https://s3-us-west-2.amazonaws.com/kafka-packages" RUN mkdir -p "/opt/kafka-0.8.2.2" && chmod a+rw /opt/kafka-0.8.2.2 && curl -s "$KAFKA_MIRROR/kafka_2.11-0.8.2.2.tgz" | tar xz --strip-components=1 -C "/opt/kafka-0.8.2.2" RUN mkdir -p "/opt/kafka-0.9.0.1" && chmod a+rw /opt/kafka-0.9.0.1 && curl -s "$KAFKA_MIRROR/kafka_2.11-0.9.0.1.tgz" | tar xz --strip-components=1 -C "/opt/kafka-0.9.0.1" -RUN mkdir -p "/opt/kafka-0.10.0.0" && chmod a+rw /opt/kafka-0.10.0.0 && curl -s "$KAFKA_MIRROR/kafka_2.11-0.10.0.0.tgz" | tar xz --strip-components=1 -C "/opt/kafka-0.10.0.0" RUN mkdir -p "/opt/kafka-0.10.0.1" && chmod a+rw /opt/kafka-0.10.0.1 && curl -s "$KAFKA_MIRROR/kafka_2.11-0.10.0.1.tgz" | tar xz --strip-components=1 -C "/opt/kafka-0.10.0.1" -RUN mkdir -p "/opt/kafka-0.10.1.0" && chmod a+rw /opt/kafka-0.10.1.0 && curl -s "$KAFKA_MIRROR/kafka_2.11-0.10.1.0.tgz" | tar xz --strip-components=1 -C "/opt/kafka-0.10.1.0" RUN mkdir -p "/opt/kafka-0.10.1.1" && chmod a+rw /opt/kafka-0.10.1.1 && curl -s "$KAFKA_MIRROR/kafka_2.11-0.10.1.1.tgz" | tar xz --strip-components=1 -C "/opt/kafka-0.10.1.1" -RUN mkdir -p "/opt/kafka-0.10.2.0" && chmod a+rw /opt/kafka-0.10.2.0 && curl -s "$KAFKA_MIRROR/kafka_2.11-0.10.2.0.tgz" | tar xz --strip-components=1 -C "/opt/kafka-0.10.2.0" -RUN mkdir -p "/opt/kafka-0.10.2.1" && chmod a+rw /opt/kafka-0.10.2.1 && curl -s "$KAFKA_MIRROR/kafka_2.11-0.10.2.1.tgz" | tar xz --strip-components=1 -C "/opt/kafka-0.10.2.1" -RUN mkdir -p "/opt/kafka-0.11.0.0" && chmod a+rw /opt/kafka-0.11.0.0 && curl -s "$KAFKA_MIRROR/kafka_2.11-0.11.0.0.tgz" | tar xz --strip-components=1 -C "/opt/kafka-0.11.0.0" -RUN mkdir -p "/opt/kafka-0.11.0.1" && chmod a+rw /opt/kafka-0.11.0.1 && curl -s "$KAFKA_MIRROR/kafka_2.11-0.11.0.1.tgz" | tar xz --strip-components=1 -C "/opt/kafka-0.11.0.1" -RUN mkdir -p "/opt/kafka-0.11.0.2" && chmod a+rw /opt/kafka-0.11.0.2 && curl -s "$KAFKA_MIRROR/kafka_2.11-0.11.0.2.tgz" | tar xz --strip-components=1 -C "/opt/kafka-0.11.0.2" -RUN mkdir -p "/opt/kafka-1.0.0" && chmod a+rw /opt/kafka-1.0.0 && curl -s "$KAFKA_MIRROR/kafka_2.11-1.0.0.tgz" | tar xz --strip-components=1 -C "/opt/kafka-1.0.0" -RUN mkdir -p "/opt/kafka-1.0.1" && chmod a+rw /opt/kafka-1.0.1 && curl -s "$KAFKA_MIRROR/kafka_2.11-1.0.1.tgz" | tar xz --strip-components=1 -C "/opt/kafka-1.0.1" -RUN mkdir -p "/opt/kafka-1.1.0" && chmod a+rw /opt/kafka-1.1.0 && curl -s "$KAFKA_MIRROR/kafka_2.11-1.1.0.tgz" | tar xz --strip-components=1 -C "/opt/kafka-1.1.0" +RUN mkdir -p "/opt/kafka-0.10.2.2" && chmod a+rw /opt/kafka-0.10.2.2 && curl -s "$KAFKA_MIRROR/kafka_2.11-0.10.2.2.tgz" | tar xz --strip-components=1 -C "/opt/kafka-0.10.2.2" +RUN mkdir -p "/opt/kafka-0.11.0.3" && chmod a+rw /opt/kafka-0.11.0.3 && curl -s "$KAFKA_MIRROR/kafka_2.11-0.11.0.3.tgz" | tar xz --strip-components=1 -C "/opt/kafka-0.11.0.3" +RUN mkdir -p "/opt/kafka-1.0.2" && chmod a+rw /opt/kafka-1.0.2 && curl -s "$KAFKA_MIRROR/kafka_2.11-1.0.2.tgz" | tar xz --strip-components=1 -C "/opt/kafka-1.0.2" +RUN mkdir -p "/opt/kafka-1.1.1" && chmod a+rw /opt/kafka-1.1.1 && curl -s "$KAFKA_MIRROR/kafka_2.11-1.1.1.tgz" | tar xz --strip-components=1 -C "/opt/kafka-1.1.1" # Streams test dependencies -RUN curl -s "$KAFKA_MIRROR/kafka-streams-0.10.0.0-test.jar" -o /opt/kafka-0.10.0.0/libs/kafka-streams-0.10.0.0-test.jar RUN curl -s "$KAFKA_MIRROR/kafka-streams-0.10.0.1-test.jar" -o /opt/kafka-0.10.0.1/libs/kafka-streams-0.10.0.1-test.jar -RUN curl -s "$KAFKA_MIRROR/kafka-streams-0.10.1.0-test.jar" -o /opt/kafka-0.10.1.0/libs/kafka-streams-0.10.1.0-test.jar RUN curl -s "$KAFKA_MIRROR/kafka-streams-0.10.1.1-test.jar" -o /opt/kafka-0.10.1.1/libs/kafka-streams-0.10.1.1-test.jar -RUN curl -s "$KAFKA_MIRROR/kafka-streams-0.10.2.0-test.jar" -o /opt/kafka-0.10.2.0/libs/kafka-streams-0.10.2.0-test.jar -RUN curl -s "$KAFKA_MIRROR/kafka-streams-0.10.2.1-test.jar" -o /opt/kafka-0.10.2.1/libs/kafka-streams-0.10.2.1-test.jar -RUN curl -s "$KAFKA_MIRROR/kafka-streams-0.11.0.0-test.jar" -o /opt/kafka-0.11.0.0/libs/kafka-streams-0.11.0.0-test.jar -RUN curl -s "$KAFKA_MIRROR/kafka-streams-0.11.0.1-test.jar" -o /opt/kafka-0.11.0.1/libs/kafka-streams-0.11.0.1-test.jar -RUN curl -s "$KAFKA_MIRROR/kafka-streams-0.11.0.2-test.jar" -o /opt/kafka-0.11.0.2/libs/kafka-streams-0.11.0.2-test.jar -RUN curl -s "$KAFKA_MIRROR/kafka-streams-1.0.0-test.jar" -o /opt/kafka-1.0.0/libs/kafka-streams-1.0.0-test.jar -RUN curl -s "$KAFKA_MIRROR/kafka-streams-1.0.1-test.jar" -o /opt/kafka-1.0.1/libs/kafka-streams-1.0.1-test.jar -RUN curl -s "$KAFKA_MIRROR/kafka-streams-1.1.0-test.jar" -o /opt/kafka-1.1.0/libs/kafka-streams-1.1.0-test.jar +RUN curl -s "$KAFKA_MIRROR/kafka-streams-0.10.2.2-test.jar" -o /opt/kafka-0.10.2.2/libs/kafka-streams-0.10.2.2-test.jar +RUN curl -s "$KAFKA_MIRROR/kafka-streams-0.11.0.3-test.jar" -o /opt/kafka-0.11.0.3/libs/kafka-streams-0.11.0.3-test.jar +RUN curl -s "$KAFKA_MIRROR/kafka-streams-1.0.2-test.jar" -o /opt/kafka-1.0.2/libs/kafka-streams-1.0.2-test.jar +RUN curl -s "$KAFKA_MIRROR/kafka-streams-1.1.1-test.jar" -o /opt/kafka-1.1.1/libs/kafka-streams-1.1.1-test.jar # The version of Kibosh to use for testing. # If you update this, also update vagrant/base.sy diff --git a/tests/docker/ducker-ak b/tests/docker/ducker-ak index f7eae494362f9..ba8ccf4d2988a 100755 --- a/tests/docker/ducker-ak +++ b/tests/docker/ducker-ak @@ -69,8 +69,8 @@ up [-n|--num-nodes NUM_NODES] [-f|--force] [docker-image] test [test-name(s)] Run a test or set of tests inside the currently active Ducker nodes. - For example, to run the Muckrake test simple_consumer_shell_test, you would run: - ./tests/docker/ducker-ak test ./tests/kafkatest/test/core/simple_consumer_shell_test.py + For example, to run the system test produce_bench_test, you would run: + ./tests/docker/ducker-ak test ./tests/kafkatest/test/core/produce_bench_test.py ssh [node-name|user-name@node-name] [command] Log in to a running ducker container. If node-name is not given, it prints diff --git a/tests/kafkatest/__init__.py b/tests/kafkatest/__init__.py index 8f7aadd12e5f6..1f3224fc5a201 100644 --- a/tests/kafkatest/__init__.py +++ b/tests/kafkatest/__init__.py @@ -22,4 +22,4 @@ # Instead, in development branches, the version should have a suffix of the form ".devN" # # For example, when Kafka is at version 1.0.0-SNAPSHOT, this should be something like "1.0.0.dev0" -__version__ = '2.0.0.dev0' +__version__ = '2.0.2.dev0' diff --git a/tests/kafkatest/benchmarks/core/benchmark_test.py b/tests/kafkatest/benchmarks/core/benchmark_test.py index b068fff63caf4..2b4ff87bb35db 100644 --- a/tests/kafkatest/benchmarks/core/benchmark_test.py +++ b/tests/kafkatest/benchmarks/core/benchmark_test.py @@ -189,16 +189,14 @@ def test_end_to_end_latency(self, compression_type="none", security_protocol="PL return latency(self.perf.results[0]['latency_50th_ms'], self.perf.results[0]['latency_99th_ms'], self.perf.results[0]['latency_999th_ms']) @cluster(num_nodes=6) - @parametrize(security_protocol='PLAINTEXT', new_consumer=False) @parametrize(security_protocol='SSL', interbroker_security_protocol='PLAINTEXT') @matrix(security_protocol=['PLAINTEXT', 'SSL'], compression_type=["none", "snappy"]) def test_producer_and_consumer(self, compression_type="none", security_protocol="PLAINTEXT", - interbroker_security_protocol=None, new_consumer=True, + interbroker_security_protocol=None, client_version=str(DEV_BRANCH), broker_version=str(DEV_BRANCH)): """ 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. @@ -224,7 +222,7 @@ def test_producer_and_consumer(self, compression_type="none", security_protocol= } ) self.consumer = ConsumerPerformanceService( - self.test_context, 1, self.kafka, topic=TOPIC_REP_THREE, new_consumer=new_consumer, messages=num_records) + self.test_context, 1, self.kafka, topic=TOPIC_REP_THREE, messages=num_records) Service.run_parallel(self.producer, self.consumer) data = { @@ -238,15 +236,14 @@ def test_producer_and_consumer(self, compression_type="none", security_protocol= return data @cluster(num_nodes=6) - @parametrize(security_protocol='PLAINTEXT', new_consumer=False) @parametrize(security_protocol='SSL', interbroker_security_protocol='PLAINTEXT') @matrix(security_protocol=['PLAINTEXT', 'SSL'], compression_type=["none", "snappy"]) def test_consumer_throughput(self, compression_type="none", security_protocol="PLAINTEXT", - interbroker_security_protocol=None, new_consumer=True, num_consumers=1, + interbroker_security_protocol=None, num_consumers=1, client_version=str(DEV_BRANCH), broker_version=str(DEV_BRANCH)): """ 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. + and report throughput. """ client_version = KafkaVersion(client_version) broker_version = KafkaVersion(broker_version) @@ -273,7 +270,7 @@ def test_consumer_throughput(self, compression_type="none", security_protocol="P # consume self.consumer = ConsumerPerformanceService( self.test_context, num_consumers, self.kafka, - topic=TOPIC_REP_THREE, new_consumer=new_consumer, messages=num_records) + topic=TOPIC_REP_THREE, messages=num_records) self.consumer.group = "test-consumer-group" self.consumer.run() return compute_aggregate_throughput(self.consumer) diff --git a/tests/kafkatest/sanity_checks/test_console_consumer.py b/tests/kafkatest/sanity_checks/test_console_consumer.py index 537755d5820f1..061bc3a77bbf5 100644 --- a/tests/kafkatest/sanity_checks/test_console_consumer.py +++ b/tests/kafkatest/sanity_checks/test_console_consumer.py @@ -38,18 +38,17 @@ def __init__(self, test_context): self.zk = ZookeeperService(test_context, num_nodes=1) self.kafka = KafkaService(self.test_context, num_nodes=1, zk=self.zk, zk_chroot="/kafka", topics={self.topic: {"partitions": 1, "replication-factor": 1}}) - self.consumer = ConsoleConsumer(self.test_context, num_nodes=1, kafka=self.kafka, topic=self.topic, new_consumer=False) + self.consumer = ConsoleConsumer(self.test_context, num_nodes=1, kafka=self.kafka, topic=self.topic) def setUp(self): self.zk.start() @cluster(num_nodes=3) - @parametrize(security_protocol='PLAINTEXT', new_consumer=False) @matrix(security_protocol=['PLAINTEXT', 'SSL']) @cluster(num_nodes=4) @matrix(security_protocol=['SASL_SSL'], sasl_mechanism=['PLAIN', 'SCRAM-SHA-256', 'SCRAM-SHA-512']) @matrix(security_protocol=['SASL_PLAINTEXT', 'SASL_SSL']) - def test_lifecycle(self, security_protocol, new_consumer=True, sasl_mechanism='GSSAPI'): + def test_lifecycle(self, security_protocol, sasl_mechanism='GSSAPI'): """Check that console consumer starts/stops properly, and that we are capturing log output.""" self.kafka.security_protocol = security_protocol @@ -58,7 +57,6 @@ def test_lifecycle(self, security_protocol, new_consumer=True, sasl_mechanism='G self.kafka.start() self.consumer.security_protocol = security_protocol - self.consumer.new_consumer = new_consumer t0 = time.time() self.consumer.start() @@ -91,6 +89,7 @@ def test_version(self): self.producer.wait() self.consumer.nodes[0].version = LATEST_0_8_2 + self.consumer.new_consumer = False self.consumer.consumer_timeout_ms = 1000 self.consumer.start() self.consumer.wait() diff --git a/tests/kafkatest/sanity_checks/test_performance_services.py b/tests/kafkatest/sanity_checks/test_performance_services.py index 6ff34b8606bee..280152c0f8621 100644 --- a/tests/kafkatest/sanity_checks/test_performance_services.py +++ b/tests/kafkatest/sanity_checks/test_performance_services.py @@ -21,7 +21,7 @@ from kafkatest.services.performance import ProducerPerformanceService, ConsumerPerformanceService, EndToEndLatencyService from kafkatest.services.performance import latency, compute_aggregate_throughput from kafkatest.services.zookeeper import ZookeeperService -from kafkatest.version import DEV_BRANCH, LATEST_0_8_2, LATEST_0_9, KafkaVersion +from kafkatest.version import DEV_BRANCH, LATEST_0_8_2, LATEST_0_9, LATEST_1_1, KafkaVersion class PerformanceServiceTest(Test): @@ -42,7 +42,7 @@ def setUp(self): @parametrize(version=str(LATEST_0_8_2), new_consumer=False) @parametrize(version=str(LATEST_0_9), new_consumer=False) @parametrize(version=str(LATEST_0_9)) - @parametrize(version=str(DEV_BRANCH), new_consumer=False) + @parametrize(version=str(LATEST_1_1), new_consumer=False) @parametrize(version=str(DEV_BRANCH)) def test_version(self, version=str(LATEST_0_9), new_consumer=True): """ diff --git a/tests/kafkatest/services/connect.py b/tests/kafkatest/services/connect.py index d7ef204b8a21e..96e0d54c87a89 100644 --- a/tests/kafkatest/services/connect.py +++ b/tests/kafkatest/services/connect.py @@ -20,7 +20,6 @@ import time import requests -from ducktape.cluster.remoteaccount import RemoteCommandError from ducktape.errors import DucktapeError from ducktape.services.service import Service from ducktape.utils.util import wait_until @@ -40,7 +39,9 @@ class ConnectServiceBase(KafkaPathResolverMixin, Service): STDERR_FILE = os.path.join(PERSISTENT_ROOT, "connect.stderr") LOG4J_CONFIG_FILE = os.path.join(PERSISTENT_ROOT, "connect-log4j.properties") PID_FILE = os.path.join(PERSISTENT_ROOT, "connect.pid") + EXTERNAL_CONFIGS_FILE = os.path.join(PERSISTENT_ROOT, "connect-external-configs.properties") CONNECT_REST_PORT = 8083 + HEAP_DUMP_FILE = os.path.join(PERSISTENT_ROOT, "connect_heap_dump.bin") # Currently the Connect worker supports waiting on three modes: STARTUP_MODE_INSTANT = 'INSTANT' @@ -60,15 +61,20 @@ class ConnectServiceBase(KafkaPathResolverMixin, Service): "connect_stderr": { "path": STDERR_FILE, "collect_default": True}, + "connect_heap_dump_file": { + "path": HEAP_DUMP_FILE, + "collect_default": True} } - def __init__(self, context, num_nodes, kafka, files): + def __init__(self, context, num_nodes, kafka, files, startup_timeout_sec = 60): super(ConnectServiceBase, self).__init__(context, num_nodes) self.kafka = kafka self.security_config = kafka.security_config.client_config() self.files = files self.startup_mode = self.STARTUP_MODE_LISTEN + self.startup_timeout_sec = startup_timeout_sec self.environment = {} + self.external_config_template_func = None def pids(self, node): """Return process ids for Kafka Connect processes.""" @@ -87,14 +93,25 @@ def set_configs(self, config_template_func, connector_config_templates=None): self.config_template_func = config_template_func self.connector_config_templates = connector_config_templates + def set_external_configs(self, external_config_template_func): + """ + Set the properties that will be written in the external file properties + as used by the org.apache.kafka.common.config.provider.FileConfigProvider. + When this is used, the worker configuration must also enable the FileConfigProvider. + This is not provided in the constructor in case the worker + config generally needs access to ZK/Kafka services to + create the configuration. + """ + self.external_config_template_func = external_config_template_func + def listening(self, node): try: - cmd = "nc -z %s %s" % (node.account.hostname, self.CONNECT_REST_PORT) - node.account.ssh_output(cmd, allow_fail=False) - self.logger.debug("Connect worker started accepting connections at: '%s:%s')", node.account.hostname, + self.list_connectors(node) + self.logger.debug("Connect worker started serving REST at: '%s:%s')", node.account.hostname, self.CONNECT_REST_PORT) return True - except (RemoteCommandError, ValueError) as e: + except requests.exceptions.ConnectionError: + self.logger.debug("REST resources are not loaded yet") return False def start(self, mode=STARTUP_MODE_LISTEN): @@ -109,13 +126,13 @@ def start_and_return_immediately(self, node, worker_type, remote_connector_confi def start_and_wait_to_load_plugins(self, node, worker_type, remote_connector_configs): with node.account.monitor_log(self.LOG_FILE) as monitor: self.start_and_return_immediately(node, worker_type, remote_connector_configs) - monitor.wait_until('Kafka version', timeout_sec=60, + monitor.wait_until('Kafka version', timeout_sec=self.startup_timeout_sec, err_msg="Never saw message indicating Kafka Connect finished startup on node: " + "%s in condition mode: %s" % (str(node.account), self.startup_mode)) def start_and_wait_to_start_listening(self, node, worker_type, remote_connector_configs): self.start_and_return_immediately(node, worker_type, remote_connector_configs) - wait_until(lambda: self.listening(node), timeout_sec=60, + wait_until(lambda: self.listening(node), timeout_sec=self.startup_timeout_sec, err_msg="Kafka Connect failed to start on node: %s in condition mode: %s" % (str(node.account), self.startup_mode)) @@ -128,7 +145,8 @@ def stop_node(self, node, clean_shutdown=True): node.account.signal(pid, sig, allow_fail=True) if clean_shutdown: for pid in pids: - wait_until(lambda: not node.account.alive(pid), timeout_sec=60, err_msg="Kafka Connect process on " + str(node.account) + " took too long to exit") + wait_until(lambda: not node.account.alive(pid), timeout_sec=self.startup_timeout_sec, err_msg="Kafka Connect process on " + str( + node.account) + " took too long to exit") node.account.ssh("rm -f " + self.PID_FILE, allow_fail=False) @@ -145,8 +163,8 @@ def restart_node(self, node, clean_shutdown=True): def clean_node(self, node): node.account.kill_process("connect", clean_shutdown=False, allow_fail=True) self.security_config.clean_node(node) - all_files = " ".join([self.CONFIG_FILE, self.LOG4J_CONFIG_FILE, self.PID_FILE, self.LOG_FILE, self.STDOUT_FILE, self.STDERR_FILE] + self.config_filenames() + self.files) - node.account.ssh("rm -rf " + all_files, allow_fail=False) + other_files = " ".join(self.config_filenames() + self.files) + node.account.ssh("rm -rf -- %s %s" % (ConnectServiceBase.PERSISTENT_ROOT, other_files), allow_fail=False) def config_filenames(self): return [os.path.join(self.PERSISTENT_ROOT, "connect-connector-" + str(idx) + ".properties") for idx, template in enumerate(self.connector_config_templates or [])] @@ -221,7 +239,7 @@ def _rest(self, path, body=None, node=None, method="GET"): def _rest_with_retry(self, path, body=None, node=None, method="GET", retries=40, retry_backoff=.25): """ Invokes a REST API with retries for errors that may occur during normal operation (notably 409 CONFLICT - responses that can occur due to rebalancing). + responses that can occur due to rebalancing or 404 when the connect resources are not initialized yet). """ exception_to_throw = None for i in range(0, retries + 1): @@ -229,7 +247,7 @@ def _rest_with_retry(self, path, body=None, node=None, method="GET", retries=40, return self._rest(path, body, node, method) except ConnectRestError as e: exception_to_throw = e - if e.status != 409: + if e.status != 409 and e.status != 404: break time.sleep(retry_backoff) raise exception_to_throw @@ -237,12 +255,20 @@ def _rest_with_retry(self, path, body=None, node=None, method="GET", retries=40, def _base_url(self, node): return 'http://' + node.account.externally_routable_ip + ':' + str(self.CONNECT_REST_PORT) + def append_to_environment_variable(self, envvar, value): + env_opts = self.environment[envvar] + if env_opts is None: + env_opts = "\"%s\"" % value + else: + env_opts = "\"%s %s\"" % (env_opts.strip('\"'), value) + self.environment[envvar] = env_opts + class ConnectStandaloneService(ConnectServiceBase): """Runs Kafka Connect in standalone mode.""" - def __init__(self, context, kafka, files): - super(ConnectStandaloneService, self).__init__(context, 1, kafka, files) + def __init__(self, context, kafka, files, startup_timeout_sec = 60): + super(ConnectStandaloneService, self).__init__(context, 1, kafka, files, startup_timeout_sec) # For convenience since this service only makes sense with a single node @property @@ -251,7 +277,10 @@ def node(self): def start_cmd(self, node, connector_configs): cmd = "( export KAFKA_LOG4J_OPTS=\"-Dlog4j.configuration=file:%s\"; " % self.LOG4J_CONFIG_FILE - cmd += "export KAFKA_OPTS=%s; " % self.security_config.kafka_opts + heap_kafka_opts = "-XX:+HeapDumpOnOutOfMemoryError -XX:HeapDumpPath=%s" % \ + self.logs["connect_heap_dump_file"]["path"] + other_kafka_opts = self.security_config.kafka_opts.strip('\"') + cmd += "export KAFKA_OPTS=\"%s %s\"; " % (heap_kafka_opts, other_kafka_opts) for envvar in self.environment: cmd += "export %s=%s; " % (envvar, str(self.environment[envvar])) cmd += "%s %s " % (self.path.script("connect-standalone.sh", node), self.CONFIG_FILE) @@ -263,6 +292,8 @@ def start_node(self, node): node.account.ssh("mkdir -p %s" % self.PERSISTENT_ROOT, allow_fail=False) self.security_config.setup_node(node) + if self.external_config_template_func: + node.account.create_file(self.EXTERNAL_CONFIGS_FILE, self.external_config_template_func(node)) node.account.create_file(self.CONFIG_FILE, self.config_template_func(node)) node.account.create_file(self.LOG4J_CONFIG_FILE, self.render('connect_log4j.properties', log_file=self.LOG_FILE)) remote_connector_configs = [] @@ -288,8 +319,8 @@ class ConnectDistributedService(ConnectServiceBase): """Runs Kafka Connect in distributed mode.""" def __init__(self, context, num_nodes, kafka, files, offsets_topic="connect-offsets", - configs_topic="connect-configs", status_topic="connect-status"): - super(ConnectDistributedService, self).__init__(context, num_nodes, kafka, files) + configs_topic="connect-configs", status_topic="connect-status", startup_timeout_sec = 60): + super(ConnectDistributedService, self).__init__(context, num_nodes, kafka, files, startup_timeout_sec) self.offsets_topic = offsets_topic self.configs_topic = configs_topic self.status_topic = status_topic @@ -297,7 +328,10 @@ def __init__(self, context, num_nodes, kafka, files, offsets_topic="connect-offs # connector_configs argument is intentionally ignored in distributed service. def start_cmd(self, node, connector_configs): cmd = "( export KAFKA_LOG4J_OPTS=\"-Dlog4j.configuration=file:%s\"; " % self.LOG4J_CONFIG_FILE - cmd += "export KAFKA_OPTS=%s; " % self.security_config.kafka_opts + heap_kafka_opts = "-XX:+HeapDumpOnOutOfMemoryError -XX:HeapDumpPath=%s" % \ + self.logs["connect_heap_dump_file"]["path"] + other_kafka_opts = self.security_config.kafka_opts.strip('\"') + cmd += "export KAFKA_OPTS=\"%s %s\"; " % (heap_kafka_opts, other_kafka_opts) for envvar in self.environment: cmd += "export %s=%s; " % (envvar, str(self.environment[envvar])) cmd += "%s %s " % (self.path.script("connect-distributed.sh", node), self.CONFIG_FILE) @@ -308,6 +342,8 @@ def start_node(self, node): node.account.ssh("mkdir -p %s" % self.PERSISTENT_ROOT, allow_fail=False) self.security_config.setup_node(node) + if self.external_config_template_func: + node.account.create_file(self.EXTERNAL_CONFIGS_FILE, self.external_config_template_func(node)) node.account.create_file(self.CONFIG_FILE, self.config_template_func(node)) node.account.create_file(self.LOG4J_CONFIG_FILE, self.render('connect_log4j.properties', log_file=self.LOG_FILE)) if self.connector_config_templates: @@ -326,6 +362,11 @@ def start_node(self, node): raise RuntimeError("No process ids recorded") +class ErrorTolerance(object): + ALL = "all" + NONE = "none" + + class ConnectRestError(RuntimeError): def __init__(self, status, msg, url): self.status = status diff --git a/tests/kafkatest/services/console_consumer.py b/tests/kafkatest/services/console_consumer.py index 64a99f938e6a2..51b70befc4129 100644 --- a/tests/kafkatest/services/console_consumer.py +++ b/tests/kafkatest/services/console_consumer.py @@ -17,50 +17,13 @@ import os from ducktape.services.background_thread import BackgroundThreadService -from ducktape.cluster.remoteaccount import RemoteCommandError from kafkatest.directory_layout.kafka_path import KafkaPathResolverMixin from kafkatest.services.monitor.jmx import JmxMixin -from kafkatest.version import DEV_BRANCH, LATEST_0_8_2, LATEST_0_9, LATEST_0_10_0, V_0_10_0_0, V_0_11_0_0 +from kafkatest.version import DEV_BRANCH, LATEST_0_8_2, LATEST_0_9, LATEST_0_10_0, V_0_9_0_0, V_0_10_0_0, V_0_11_0_0, V_2_0_0 """ -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. """ @@ -97,7 +60,7 @@ class ConsoleConsumer(KafkaPathResolverMixin, JmxMixin, BackgroundThreadService) def __init__(self, context, num_nodes, kafka, topic, group_id="test-consumer-group", new_consumer=True, message_validator=None, from_beginning=True, consumer_timeout_ms=None, version=DEV_BRANCH, client_id="console-consumer", print_key=False, jmx_object_names=None, jmx_attributes=None, - enable_systest_events=False, stop_timeout_sec=15, print_timestamp=False, + enable_systest_events=False, stop_timeout_sec=30, print_timestamp=False, isolation_level="read_uncommitted"): """ Args: @@ -191,13 +154,18 @@ def start_cmd(self, node): "--topic %(topic)s --consumer.config %(config_file)s" % args if self.new_consumer: + assert node.version >= V_0_9_0_0, \ + "new_consumer is only supported if version >= 0.9.0.0, version %s" % str(node.version) if node.version <= LATEST_0_10_0: cmd += " --new-consumer" cmd += " --bootstrap-server %(broker_list)s" % args if node.version >= V_0_11_0_0: cmd += " --isolation-level %s" % self.isolation_level else: + assert node.version < V_2_0_0, \ + "new_consumer==false is only supported if version < 2.0.0, version %s" % str(node.version) cmd += " --zookeeper %(zk_connect)s" % args + if self.from_beginning: cmd += " --from-beginning" @@ -227,12 +195,7 @@ def start_cmd(self, node): 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 (RemoteCommandError, ValueError) as e: - return [] + return node.account.java_pids(self.java_class_name()) def alive(self, node): return len(self.pids(node)) > 0 @@ -282,7 +245,9 @@ def start_node(self, node): BackgroundThreadService.start_node(self, node) def stop_node(self, node): - node.account.kill_process("console_consumer", allow_fail=True) + self.logger.info("%s Stopping node %s" % (self.__class__.__name__, str(node.account))) + node.account.kill_java_processes(self.java_class_name(), + clean_shutdown=True, allow_fail=True) stopped = self.wait_node(node, timeout_sec=self.stop_timeout_sec) assert stopped, "Node %s: did not stop within the specified timeout of %s seconds" % \ @@ -293,10 +258,13 @@ def clean_node(self, node): self.logger.warn("%s %s was still alive at cleanup time. Killing forcefully..." % (self.__class__.__name__, node.account)) JmxMixin.clean_node(self, node) - node.account.kill_process("java", clean_shutdown=False, allow_fail=True) + node.account.kill_java_processes(self.java_class_name(), clean_shutdown=False, allow_fail=True) node.account.ssh("rm -rf %s" % ConsoleConsumer.PERSISTENT_ROOT, allow_fail=False) self.security_config.clean_node(node) + def java_class_name(self): + return "ConsoleConsumer" + def has_partitions_assigned(self, node): if self.new_consumer is False: return False diff --git a/tests/kafkatest/services/kafka/config_property.py b/tests/kafkatest/services/kafka/config_property.py index b261960c6530f..621b8e5b55bfe 100644 --- a/tests/kafkatest/services/kafka/config_property.py +++ b/tests/kafkatest/services/kafka/config_property.py @@ -48,6 +48,7 @@ LOG_FLUSH_INTERVAL_MESSAGE = "log.flush.interval.messages" REPLICA_HIGHWATERMARK_CHECKPOINT_INTERVAL_MS = "replica.high.watermark.checkpoint.interval.ms" LOG_ROLL_TIME_MS = "log.roll.ms" +OFFSETS_TOPIC_NUM_PARTITIONS = "offsets.topic.num.partitions" """ From KafkaConfig.scala diff --git a/tests/kafkatest/services/kafka/kafka.py b/tests/kafkatest/services/kafka/kafka.py index 7e919b37249a5..a59bb71f62fd7 100644 --- a/tests/kafkatest/services/kafka/kafka.py +++ b/tests/kafkatest/services/kafka/kafka.py @@ -49,6 +49,7 @@ class KafkaService(KafkaPathResolverMixin, JmxMixin, Service): CONFIG_FILE = os.path.join(PERSISTENT_ROOT, "kafka.properties") # Kafka Authorizer SIMPLE_AUTHORIZER = "kafka.security.auth.SimpleAclAuthorizer" + HEAP_DUMP_FILE = os.path.join(PERSISTENT_ROOT, "kafka_heap_dump.bin") logs = { "kafka_server_start_stdout_stderr": { @@ -65,7 +66,10 @@ class KafkaService(KafkaPathResolverMixin, JmxMixin, Service): "collect_default": False}, "kafka_data_2": { "path": DATA_LOG_DIR_2, - "collect_default": False} + "collect_default": False}, + "kafka_heap_dump_file": { + "path": HEAP_DUMP_FILE, + "collect_default": True} } def __init__(self, context, num_nodes, zk, security_protocol=SecurityConfig.PLAINTEXT, interbroker_security_protocol=SecurityConfig.PLAINTEXT, @@ -211,25 +215,46 @@ def set_protocol_and_port(self, node): self.advertised_listeners = ','.join(advertised_listeners) def prop_file(self, node): - cfg = KafkaConfig(**node.config) - cfg[config_property.ADVERTISED_HOSTNAME] = node.account.hostname - cfg[config_property.ZOOKEEPER_CONNECT] = self.zk_connect_setting() + self.set_protocol_and_port(node) + + #load template configs as dictionary + config_template = self.render('kafka.properties', node=node, broker_id=self.idx(node), + security_config=self.security_config, num_nodes=self.num_nodes) + + configs = dict( l.rstrip().split('=', 1) for l in config_template.split('\n') + if not l.startswith("#") and "=" in l ) + + #load specific test override configs + override_configs = KafkaConfig(**node.config) + override_configs[config_property.ADVERTISED_HOSTNAME] = node.account.hostname + override_configs[config_property.ZOOKEEPER_CONNECT] = self.zk_connect_setting() for prop in self.server_prop_overides: - cfg[prop[0]] = prop[1] + override_configs[prop[0]] = prop[1] - self.set_protocol_and_port(node) + #update template configs with test override configs + configs.update(override_configs) - # TODO - clean up duplicate configuration logic - prop_file = cfg.render() - prop_file += self.render('kafka.properties', node=node, broker_id=self.idx(node), - security_config=self.security_config, num_nodes=self.num_nodes) + prop_file = self.render_configs(configs) return prop_file + def render_configs(self, configs): + """Render self as a series of lines key=val\n, and do so in a consistent order. """ + keys = [k for k in configs.keys()] + keys.sort() + + s = "" + for k in keys: + s += "%s=%s\n" % (k, str(configs[k])) + return s + def start_cmd(self, node): cmd = "export JMX_PORT=%d; " % self.jmx_port cmd += "export KAFKA_LOG4J_OPTS=\"-Dlog4j.configuration=file:%s\"; " % self.LOG4J_CONFIG - cmd += "export KAFKA_OPTS=%s; " % self.security_config.kafka_opts + heap_kafka_opts = "-XX:+HeapDumpOnOutOfMemoryError -XX:HeapDumpPath=%s" % \ + self.logs["kafka_heap_dump_file"]["path"] + other_kafka_opts = self.security_config.kafka_opts.strip('\"') + cmd += "export KAFKA_OPTS=\"%s %s\"; " % (heap_kafka_opts, other_kafka_opts) cmd += "%s %s 1>> %s 2>> %s &" % \ (self.path.script("kafka-server-start.sh", node), KafkaService.CONFIG_FILE, @@ -253,7 +278,7 @@ def start_node(self, node): with node.account.monitor_log(KafkaService.STDOUT_STDERR_CAPTURE) as monitor: node.account.ssh(cmd) # Kafka 1.0.0 and higher don't have a space between "Kafka" and "Server" - monitor.wait_until("Kafka\s*Server.*started", timeout_sec=30, backoff_sec=.25, err_msg="Kafka server didn't finish startup") + monitor.wait_until("Kafka\s*Server.*started", timeout_sec=60, backoff_sec=.25, err_msg="Kafka server didn't finish startup") # Credentials for inter-broker communication are created before starting Kafka. # Client credentials are created after starting Kafka so that both loading of @@ -288,7 +313,19 @@ def stop_node(self, node, clean_shutdown=True): for pid in pids: node.account.signal(pid, sig, allow_fail=False) - wait_until(lambda: len(self.pids(node)) == 0, timeout_sec=60, err_msg="Kafka node failed to stop") + + try: + wait_until(lambda: len(self.pids(node)) == 0, timeout_sec=60, err_msg="Kafka node failed to stop") + except Exception: + self.thread_dump(node) + raise + + def thread_dump(self, node): + for pid in self.pids(node): + try: + node.account.signal(pid, signal.SIGQUIT, allow_fail=True) + except: + self.logger.warn("Could not dump threads on node") def clean_node(self, node): JmxMixin.clean_node(self, node) @@ -502,6 +539,10 @@ def search_data_files(self, topic, messages): return missing + def restart_cluster(self, clean_shutdown=True): + for node in self.nodes: + self.restart_node(node, clean_shutdown=clean_shutdown) + def restart_node(self, node, clean_shutdown=True): """Restart the given node.""" self.stop_node(node, clean_shutdown) @@ -571,7 +612,7 @@ def cluster_id(self): self.logger.debug("Data in /cluster/id znode could not be parsed. Data = %s" % cluster) raise - def list_consumer_groups(self, node=None, new_consumer=True, command_config=None): + def list_consumer_groups(self, node=None, command_config=None): """ Get list of consumer groups. """ if node is None: @@ -583,17 +624,10 @@ def list_consumer_groups(self, node=None, new_consumer=True, command_config=None else: command_config = "--command-config " + command_config - if new_consumer: - new_consumer_opt = "" - if node.version <= LATEST_0_10_0: - new_consumer_opt = "--new-consumer" - cmd = "%s %s --bootstrap-server %s %s --list" % \ - (consumer_group_script, - new_consumer_opt, - self.bootstrap_servers(self.security_protocol), - command_config) - else: - cmd = "%s --zookeeper %s %s --list" % (consumer_group_script, self.zk_connect_setting(), command_config) + cmd = "%s --bootstrap-server %s %s --list" % \ + (consumer_group_script, + self.bootstrap_servers(self.security_protocol), + command_config) output = "" self.logger.debug(cmd) for line in node.account.ssh_capture(cmd): @@ -602,7 +636,7 @@ def list_consumer_groups(self, node=None, new_consumer=True, command_config=None self.logger.debug(output) return output - def describe_consumer_group(self, group, node=None, new_consumer=True, command_config=None): + def describe_consumer_group(self, group, node=None, command_config=None): """ Describe a consumer group. """ if node is None: @@ -614,18 +648,11 @@ def describe_consumer_group(self, group, node=None, new_consumer=True, command_c else: command_config = "--command-config " + command_config - if new_consumer: - new_consumer_opt = "" - if node.version <= LATEST_0_10_0: - new_consumer_opt = "--new-consumer" - cmd = "%s %s --bootstrap-server %s %s --group %s --describe" % \ - (consumer_group_script, - new_consumer_opt, - self.bootstrap_servers(self.security_protocol), - command_config, group) - else: - cmd = "%s --zookeeper %s %s --group %s --describe" % \ - (consumer_group_script, self.zk_connect_setting(), command_config, group) + cmd = "%s --bootstrap-server %s %s --group %s --describe" % \ + (consumer_group_script, + self.bootstrap_servers(self.security_protocol), + command_config, group) + output = "" self.logger.debug(cmd) for line in node.account.ssh_capture(cmd): diff --git a/tests/kafkatest/services/kafka/templates/kafka.properties b/tests/kafkatest/services/kafka/templates/kafka.properties index 8cca14fa66de5..19deee30a7c6a 100644 --- a/tests/kafkatest/services/kafka/templates/kafka.properties +++ b/tests/kafkatest/services/kafka/templates/kafka.properties @@ -62,6 +62,7 @@ replica.lag.time.max.ms={{replica_lag}} {% if auto_create_topics_enable is defined and auto_create_topics_enable is not none %} auto.create.topics.enable={{ auto_create_topics_enable }} {% endif %} +offsets.topic.num.partitions={{ num_nodes }} offsets.topic.replication.factor={{ 3 if num_nodes > 3 else num_nodes }} # Set to a low, but non-zero value to exercise this path without making tests much slower group.initial.rebalance.delay.ms=100 diff --git a/tests/kafkatest/services/mirror_maker.py b/tests/kafkatest/services/mirror_maker.py index 847fa351ef749..340aa16731f75 100644 --- a/tests/kafkatest/services/mirror_maker.py +++ b/tests/kafkatest/services/mirror_maker.py @@ -17,44 +17,13 @@ from ducktape.services.service import Service from ducktape.utils.util import wait_until -from ducktape.cluster.remoteaccount import RemoteCommandError from kafkatest.directory_layout.kafka_path import KafkaPathResolverMixin """ -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. +MirrorMaker is a tool for mirroring data between two Kafka clusters. """ - class MirrorMaker(KafkaPathResolverMixin, Service): # Root directory for persistent output @@ -71,8 +40,8 @@ class MirrorMaker(KafkaPathResolverMixin, Service): "collect_default": True} } - def __init__(self, context, num_nodes, source, target, whitelist=None, blacklist=None, num_streams=1, - new_consumer=True, consumer_timeout_ms=None, offsets_storage="kafka", + def __init__(self, context, num_nodes, source, target, whitelist=None, num_streams=1, + consumer_timeout_ms=None, offsets_storage="kafka", offset_commit_interval_ms=60000, log_level="DEBUG", producer_interceptor_classes=None): """ MirrorMaker mirrors messages from one or more source clusters to a single destination cluster. @@ -92,14 +61,12 @@ def __init__(self, context, num_nodes, source, target, whitelist=None, blacklist """ super(MirrorMaker, self).__init__(context, num_nodes=num_nodes) self.log_level = log_level - self.new_consumer = new_consumer 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 @@ -135,8 +102,6 @@ def start_cmd(self, 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 diff --git a/tests/kafkatest/services/monitor/jmx.py b/tests/kafkatest/services/monitor/jmx.py index 542d3a55052bb..cf8cbc3b6c836 100644 --- a/tests/kafkatest/services/monitor/jmx.py +++ b/tests/kafkatest/services/monitor/jmx.py @@ -83,7 +83,7 @@ def check_jmx_port_listening(): self.logger.debug("%s: Start JmxTool %d command: %s" % (node.account, idx, cmd)) node.account.ssh(cmd, allow_fail=False) - wait_until(lambda: self._jmx_has_output(node), timeout_sec=10, backoff_sec=.5, err_msg="%s: Jmx tool took too long to start" % node.account) + wait_until(lambda: self._jmx_has_output(node), timeout_sec=20, backoff_sec=.5, err_msg="%s: Jmx tool took too long to start" % node.account) self.started[idx-1] = True def _jmx_has_output(self, node): diff --git a/tests/kafkatest/services/performance/consumer_performance.py b/tests/kafkatest/services/performance/consumer_performance.py index 69a8f71abd7d3..930a68fb0e91d 100644 --- a/tests/kafkatest/services/performance/consumer_performance.py +++ b/tests/kafkatest/services/performance/consumer_performance.py @@ -18,7 +18,7 @@ from kafkatest.services.performance import PerformanceService from kafkatest.services.security.security_config import SecurityConfig -from kafkatest.version import DEV_BRANCH, V_0_9_0_0, LATEST_0_10_0 +from kafkatest.version import DEV_BRANCH, V_0_9_0_0, V_2_0_0, LATEST_0_10_0 class ConsumerPerformanceService(PerformanceService): @@ -82,6 +82,9 @@ def __init__(self, context, num_nodes, kafka, topic, messages, version=DEV_BRANC assert version >= V_0_9_0_0 or (not new_consumer), \ "new_consumer is only supported if version >= 0.9.0.0, version %s" % str(version) + assert version < V_2_0_0 or new_consumer, \ + "new_consumer==false is only supported if version < 2.0.0, version %s" % str(version) + security_protocol = self.security_config.security_protocol assert version >= V_0_9_0_0 or security_protocol == SecurityConfig.PLAINTEXT, \ "Security protocol %s is only supported if version >= 0.9.0.0, version %s" % (self.security_config, str(version)) diff --git a/tests/kafkatest/services/security/minikdc.py b/tests/kafkatest/services/security/minikdc.py index 86a713882b010..f26b379a52a73 100644 --- a/tests/kafkatest/services/security/minikdc.py +++ b/tests/kafkatest/services/security/minikdc.py @@ -123,10 +123,10 @@ def start_node(self, node): def stop_node(self, node): self.logger.info("Stopping %s on %s" % (type(self).__name__, node.account.hostname)) - node.account.kill_process("apacheds", allow_fail=False) + node.account.kill_java_processes("MiniKdc", clean_shutdown=True, allow_fail=False) def clean_node(self, node): - node.account.kill_process("apacheds", clean_shutdown=False, allow_fail=False) + node.account.kill_java_processes("MiniKdc", clean_shutdown=False, allow_fail=True) node.account.ssh("rm -rf " + MiniKdc.WORK_DIR, allow_fail=False) if os.path.exists(MiniKdc.LOCAL_KEYTAB_FILE): os.remove(MiniKdc.LOCAL_KEYTAB_FILE) diff --git a/tests/kafkatest/services/simple_consumer_shell.py b/tests/kafkatest/services/simple_consumer_shell.py deleted file mode 100644 index 76820f0da59dd..0000000000000 --- a/tests/kafkatest/services/simple_consumer_shell.py +++ /dev/null @@ -1,70 +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. - -from ducktape.services.background_thread import BackgroundThreadService - -from kafkatest.directory_layout.kafka_path import KafkaPathResolverMixin - - -class SimpleConsumerShell(KafkaPathResolverMixin, BackgroundThreadService): - - logs = { - "simple_consumer_shell_log": { - "path": "/mnt/simple_consumer_shell.log", - "collect_default": False} - } - - def __init__(self, context, num_nodes, kafka, topic, partition=0, stop_timeout_sec=30): - super(SimpleConsumerShell, self).__init__(context, num_nodes) - - self.kafka = kafka - self.topic = topic - self.partition = partition - self.output = "" - self.stop_timeout_sec = stop_timeout_sec - - def _worker(self, idx, node): - cmd = self.start_cmd(node) - self.logger.debug("SimpleConsumerShell %d command: %s" % (idx, cmd)) - self.output = "" - self.logger.debug(cmd) - for line in node.account.ssh_capture(cmd): - self.output += line - self.logger.debug(self.output) - - def start_cmd(self, node): - cmd = self.path.script("kafka-run-class.sh", node) - cmd += " %s" % self.java_class_name() - cmd += " --topic %s --broker-list %s --partition %s --no-wait-at-logend" % (self.topic, self.kafka.bootstrap_servers(), self.partition) - - cmd += " 2>> /mnt/get_simple_consumer_shell.log | tee -a /mnt/get_simple_consumer_shell.log &" - return cmd - - def get_output(self): - return self.output - - def stop_node(self, node): - node.account.kill_java_processes(self.java_class_name(), allow_fail=False) - - stopped = self.wait_node(node, timeout_sec=self.stop_timeout_sec) - assert stopped, "Node %s: did not stop within the specified timeout of %s seconds" % \ - (str(node.account), str(self.stop_timeout_sec)) - - def clean_node(self, node): - node.account.kill_java_processes(self.java_class_name(), clean_shutdown=False, allow_fail=False) - node.account.ssh("rm -rf /mnt/simple_consumer_shell.log", allow_fail=False) - - def java_class_name(self): - return "kafka.tools.SimpleConsumerShell" diff --git a/tests/kafkatest/services/templates/connect_log4j.properties b/tests/kafkatest/services/templates/connect_log4j.properties index c972b1df86c39..adb35b30bfcf3 100644 --- a/tests/kafkatest/services/templates/connect_log4j.properties +++ b/tests/kafkatest/services/templates/connect_log4j.properties @@ -27,3 +27,4 @@ log4j.appender.FILE.layout.conversionPattern=[%d] %p %m (%c)%n log4j.logger.org.apache.zookeeper=ERROR log4j.logger.org.I0Itec.zkclient=ERROR +log4j.logger.org.reflections=ERROR diff --git a/tests/kafkatest/services/templates/mirror_maker_consumer.properties b/tests/kafkatest/services/templates/mirror_maker_consumer.properties index 0da386ec9bb93..2e66573074fdd 100644 --- a/tests/kafkatest/services/templates/mirror_maker_consumer.properties +++ b/tests/kafkatest/services/templates/mirror_maker_consumer.properties @@ -14,23 +14,13 @@ # limitations under the License. # see kafka.consumer.ConsumerConfig for more details -{% if new_consumer %} bootstrap.servers={{ source.bootstrap_servers(security_config.security_protocol) }} -{% else %} -zookeeper.connect={{ source.zk_connect_setting() }} -zookeeper.connection.timeout.ms={{ zookeeper_connection_timeout_ms|default(6000) }} -{% endif %} {% if source_auto_offset_reset is defined and source_auto_offset_reset is not none %} auto.offset.reset={{ source_auto_offset_reset|default('latest') }} {% endif %} group.id={{ group_id|default('test-consumer-group') }} -offsets.storage={{ offsets_storage }} - -{% if consumer_timeout_ms is defined and consumer_timeout_ms is not none %} -consumer.timeout.ms={{ consumer_timeout_ms }} -{% endif %} {% if partition_assignment_strategy is defined and partition_assignment_strategy is not none %} partition.assignment.strategy={{ partition_assignment_strategy }} diff --git a/tests/kafkatest/services/templates/mirror_maker_producer.properties b/tests/kafkatest/services/templates/mirror_maker_producer.properties index 01cb75f7df060..fcfd24bc453f2 100644 --- a/tests/kafkatest/services/templates/mirror_maker_producer.properties +++ b/tests/kafkatest/services/templates/mirror_maker_producer.properties @@ -12,13 +12,8 @@ # WITHOUT 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={{ target.bootstrap_servers(security_config.security_protocol) }} bootstrap.servers = {{ target.bootstrap_servers(security_config.security_protocol) }} -producer.type={{ producer_type|default("async") }} # sync or async -compression.codec=none -serializer.class=kafka.serializer.DefaultEncoder {% if producer_interceptor_classes is defined and producer_interceptor_classes is not none %} interceptor.classes={{ producer_interceptor_classes }} diff --git a/tests/kafkatest/services/templates/producer.properties b/tests/kafkatest/services/templates/producer.properties index ede60c8f3229e..65a48807c20c3 100644 --- a/tests/kafkatest/services/templates/producer.properties +++ b/tests/kafkatest/services/templates/producer.properties @@ -14,15 +14,4 @@ # 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/verifiable_consumer.py b/tests/kafkatest/services/verifiable_consumer.py index 95970d99becba..c0e186f465103 100644 --- a/tests/kafkatest/services/verifiable_consumer.py +++ b/tests/kafkatest/services/verifiable_consumer.py @@ -161,7 +161,8 @@ class VerifiableConsumer(KafkaPathResolverMixin, VerifiableClientMixin, Backgrou def __init__(self, context, num_nodes, kafka, topic, group_id, max_messages=-1, session_timeout_sec=30, enable_autocommit=False, assignment_strategy="org.apache.kafka.clients.consumer.RangeAssignor", - version=DEV_BRANCH, stop_timeout_sec=30, log_level="INFO"): + version=DEV_BRANCH, stop_timeout_sec=30, log_level="INFO", + on_record_consumed=None): super(VerifiableConsumer, self).__init__(context, num_nodes) self.log_level = log_level @@ -174,6 +175,7 @@ def __init__(self, context, num_nodes, kafka, topic, group_id, self.assignment_strategy = assignment_strategy self.prop_file = "" self.stop_timeout_sec = stop_timeout_sec + self.on_record_consumed = on_record_consumed self.event_handlers = {} self.global_position = {} @@ -223,6 +225,8 @@ def _worker(self, idx, node): elif name == "records_consumed": handler.handle_records_consumed(event) self._update_global_position(event, node) + elif name == "record_data" and self.on_record_consumed: + self.on_record_consumed(event, node) elif name == "partitions_revoked": handler.handle_partitions_revoked(event) elif name == "partitions_assigned": @@ -263,6 +267,8 @@ def start_cmd(self, node): cmd += " export KAFKA_OPTS=%s;" % self.security_config.kafka_opts cmd += " export KAFKA_LOG4J_OPTS=\"-Dlog4j.configuration=file:%s\"; " % VerifiableConsumer.LOG4J_CONFIG cmd += self.impl.exec_cmd(node) + if self.on_record_consumed: + cmd += " --verbose" cmd += " --group-id %s --topic %s --broker-list %s --session-timeout %s --assignment-strategy %s %s" % \ (self.group_id, self.topic, self.kafka.bootstrap_servers(self.security_config.security_protocol), self.session_timeout_sec*1000, self.assignment_strategy, "--enable-autocommit" if self.enable_autocommit else "") diff --git a/tests/kafkatest/services/verifiable_producer.py b/tests/kafkatest/services/verifiable_producer.py index cbce27e178518..fdca54f5b5c75 100644 --- a/tests/kafkatest/services/verifiable_producer.py +++ b/tests/kafkatest/services/verifiable_producer.py @@ -20,6 +20,7 @@ from ducktape.cluster.remoteaccount import RemoteCommandError from ducktape.services.background_thread import BackgroundThreadService from kafkatest.directory_layout.kafka_path import KafkaPathResolverMixin +from kafkatest.services.kafka import TopicPartition from kafkatest.services.verifiable_client import VerifiableClientMixin from kafkatest.utils import is_int, is_int_with_prefix from kafkatest.version import DEV_BRANCH @@ -84,6 +85,7 @@ def __init__(self, context, num_nodes, kafka, topic, max_messages=-1, throughput for node in self.nodes: node.version = version self.acked_values = [] + self._last_acked_offsets = {} self.not_acked_values = [] self.produced_count = {} self.clean_shutdown_nodes = set() @@ -156,7 +158,9 @@ def _worker(self, idx, node): self.produced_count[idx] += 1 elif data["name"] == "producer_send_success": + partition = TopicPartition(data["topic"], data["partition"]) self.acked_values.append(self.message_validator(data["value"])) + self._last_acked_offsets[partition] = data["offset"] self.produced_count[idx] += 1 # Log information if there is a large gap between successively acknowledged messages @@ -217,6 +221,11 @@ def pids(self, node): def alive(self, node): return len(self.pids(node)) > 0 + @property + def last_acked_offsets(self): + with self.lock: + return self._last_acked_offsets + @property def acked(self): with self.lock: @@ -245,7 +254,11 @@ def each_produced_at_least(self, count): return True def stop_node(self, node): - self.kill_node(node, clean_shutdown=True, allow_fail=False) + # There is a race condition on shutdown if using `max_messages` since the + # VerifiableProducer will shutdown automatically when all messages have been + # written. In this case, the process will be gone and the signal will fail. + allow_fail = self.max_messages > 0 + self.kill_node(node, clean_shutdown=True, allow_fail=allow_fail) stopped = self.wait_node(node, timeout_sec=self.stop_timeout_sec) assert stopped, "Node %s: did not stop within the specified timeout of %s seconds" % \ diff --git a/tests/kafkatest/services/zookeeper.py b/tests/kafkatest/services/zookeeper.py index 5bda867ed7cd6..f6a6b02c9e102 100644 --- a/tests/kafkatest/services/zookeeper.py +++ b/tests/kafkatest/services/zookeeper.py @@ -30,6 +30,7 @@ class ZookeeperService(KafkaPathResolverMixin, Service): ROOT = "/mnt/zookeeper" DATA = os.path.join(ROOT, "data") + HEAP_DUMP_FILE = os.path.join(ROOT, "zk_heap_dump.bin") logs = { "zk_log": { @@ -37,7 +38,10 @@ class ZookeeperService(KafkaPathResolverMixin, Service): "collect_default": True}, "zk_data": { "path": DATA, - "collect_default": False} + "collect_default": False}, + "zk_heap_dump_file": { + "path": HEAP_DUMP_FILE, + "collect_default": True} } def __init__(self, context, num_nodes, zk_sasl = False): @@ -76,8 +80,10 @@ def start_node(self, node): self.logger.info(config_file) node.account.create_file("%s/zookeeper.properties" % ZookeeperService.ROOT, config_file) - start_cmd = "export KAFKA_OPTS=\"%s\";" % (self.kafka_opts + ' ' + self.security_system_properties) \ + heap_kafka_opts = "-XX:+HeapDumpOnOutOfMemoryError -XX:HeapDumpPath=%s" % self.logs["zk_heap_dump_file"]["path"] + other_kafka_opts = self.kafka_opts + ' ' + self.security_system_properties \ if self.security_config.zk_sasl else self.kafka_opts + start_cmd = "export KAFKA_OPTS=\"%s %s\";" % (heap_kafka_opts, other_kafka_opts) start_cmd += "%s " % self.path.script("zookeeper-server-start.sh", node) start_cmd += "%s/zookeeper.properties &>> %s &" % (ZookeeperService.ROOT, self.logs["zk_log"]["path"]) node.account.ssh(start_cmd) diff --git a/tests/kafkatest/tests/client/client_compatibility_features_test.py b/tests/kafkatest/tests/client/client_compatibility_features_test.py index c5c2f2dcc540c..d386578719133 100644 --- a/tests/kafkatest/tests/client/client_compatibility_features_test.py +++ b/tests/kafkatest/tests/client/client_compatibility_features_test.py @@ -23,7 +23,7 @@ from kafkatest.services.zookeeper import ZookeeperService from kafkatest.services.kafka import KafkaService from ducktape.tests.test import Test -from kafkatest.version import DEV_BRANCH, LATEST_0_10_0, LATEST_0_10_1, LATEST_0_10_2, LATEST_0_11_0, V_0_11_0_0, V_0_10_1_0, KafkaVersion +from kafkatest.version import DEV_BRANCH, LATEST_0_10_0, LATEST_0_10_1, LATEST_0_10_2, LATEST_0_11_0, LATEST_1_0, LATEST_1_1, V_0_11_0_0, V_0_10_1_0, KafkaVersion def get_broker_features(broker_version): features = {} @@ -102,6 +102,8 @@ def invoke_compatibility_program(self, features): @parametrize(broker_version=str(LATEST_0_10_1)) @parametrize(broker_version=str(LATEST_0_10_2)) @parametrize(broker_version=str(LATEST_0_11_0)) + @parametrize(broker_version=str(LATEST_1_0)) + @parametrize(broker_version=str(LATEST_1_1)) def run_compatibility_test(self, broker_version): self.zk.start() self.kafka.set_version(KafkaVersion(broker_version)) diff --git a/tests/kafkatest/tests/client/client_compatibility_produce_consume_test.py b/tests/kafkatest/tests/client/client_compatibility_produce_consume_test.py index d3aa83b12e005..6e283d277dd79 100644 --- a/tests/kafkatest/tests/client/client_compatibility_produce_consume_test.py +++ b/tests/kafkatest/tests/client/client_compatibility_produce_consume_test.py @@ -22,7 +22,7 @@ from kafkatest.services.console_consumer import ConsoleConsumer from kafkatest.tests.produce_consume_validate import ProduceConsumeValidateTest from kafkatest.utils import is_int_with_prefix -from kafkatest.version import DEV_BRANCH, LATEST_0_10_0, LATEST_0_10_1, LATEST_0_10_2, LATEST_0_11_0, KafkaVersion +from kafkatest.version import DEV_BRANCH, LATEST_0_10_0, LATEST_0_10_1, LATEST_0_10_2, LATEST_0_11_0, LATEST_1_0, LATEST_1_1, KafkaVersion class ClientCompatibilityProduceConsumeTest(ProduceConsumeValidateTest): """ @@ -57,6 +57,8 @@ def min_cluster_size(self): @parametrize(broker_version=str(LATEST_0_10_1)) @parametrize(broker_version=str(LATEST_0_10_2)) @parametrize(broker_version=str(LATEST_0_11_0)) + @parametrize(broker_version=str(LATEST_1_0)) + @parametrize(broker_version=str(LATEST_1_1)) def test_produce_consume(self, broker_version): print("running producer_consumer_compat with broker_version = %s" % broker_version) self.kafka.set_version(KafkaVersion(broker_version)) diff --git a/tests/kafkatest/tests/client/compression_test.py b/tests/kafkatest/tests/client/compression_test.py index 9301de4a1bbfd..165e11add1933 100644 --- a/tests/kafkatest/tests/client/compression_test.py +++ b/tests/kafkatest/tests/client/compression_test.py @@ -54,9 +54,8 @@ def min_cluster_size(self): return super(CompressionTest, self).min_cluster_size() + self.num_producers + self.num_consumers @cluster(num_nodes=7) - @parametrize(compression_types=["snappy","gzip","lz4","none"], new_consumer=True) - @parametrize(compression_types=["snappy","gzip","lz4","none"], new_consumer=False) - def test_compressed_topic(self, compression_types, new_consumer): + @parametrize(compression_types=["snappy","gzip","lz4","none"]) + def test_compressed_topic(self, compression_types): """Test produce => consume => validate for compressed topics Setup: 1 zk, 1 kafka node, 1 topic with partitions=10, replication-factor=1 @@ -77,8 +76,7 @@ def test_compressed_topic(self, compression_types, new_consumer): message_validator=is_int_with_prefix, compression_types=compression_types) self.consumer = ConsoleConsumer(self.test_context, self.num_consumers, self.kafka, self.topic, - new_consumer=new_consumer, consumer_timeout_ms=60000, - message_validator=is_int_with_prefix) + consumer_timeout_ms=60000, message_validator=is_int_with_prefix) self.kafka.start() self.run_produce_consume_validate(lambda: wait_until( diff --git a/tests/kafkatest/tests/client/quota_test.py b/tests/kafkatest/tests/client/quota_test.py index 47a6a96b02046..4cbceefa4fba2 100644 --- a/tests/kafkatest/tests/client/quota_test.py +++ b/tests/kafkatest/tests/client/quota_test.py @@ -21,6 +21,7 @@ from kafkatest.services.kafka import KafkaService from kafkatest.services.performance import ProducerPerformanceService from kafkatest.services.console_consumer import ConsoleConsumer +from kafkatest.version import DEV_BRANCH, LATEST_1_1 class QuotaConfig(object): CLIENT_ID = 'client-id' @@ -119,7 +120,6 @@ def __init__(self, test_context): def setUp(self): self.zk.start() - self.kafka.start() def min_cluster_size(self): """Override this since we're adding services outside of the constructor""" @@ -128,15 +128,30 @@ def min_cluster_size(self): @cluster(num_nodes=5) @matrix(quota_type=[QuotaConfig.CLIENT_ID, QuotaConfig.USER, QuotaConfig.USER_CLIENT], override_quota=[True, False]) @parametrize(quota_type=QuotaConfig.CLIENT_ID, consumer_num=2) - def test_quota(self, quota_type, override_quota=True, producer_num=1, consumer_num=1): + @parametrize(quota_type=QuotaConfig.CLIENT_ID, old_broker_throttling_behavior=True) + @parametrize(quota_type=QuotaConfig.CLIENT_ID, old_client_throttling_behavior=True) + def test_quota(self, quota_type, override_quota=True, producer_num=1, consumer_num=1, + old_broker_throttling_behavior=False, old_client_throttling_behavior=False): + # Old (pre-2.0) throttling behavior for broker throttles before sending a response to the client. + if old_broker_throttling_behavior: + self.kafka.set_version(LATEST_1_1) + self.kafka.start() + self.quota_config = QuotaConfig(quota_type, override_quota, self.kafka) producer_client_id = self.quota_config.client_id consumer_client_id = self.quota_config.client_id + # Old (pre-2.0) throttling behavior for client does not throttle upon receiving a response with a non-zero throttle time. + if old_client_throttling_behavior: + client_version = LATEST_1_1 + else: + client_version = DEV_BRANCH + # Produce all messages producer = ProducerPerformanceService( self.test_context, producer_num, self.kafka, - topic=self.topic, num_records=self.num_records, record_size=self.record_size, throughput=-1, client_id=producer_client_id) + topic=self.topic, num_records=self.num_records, record_size=self.record_size, throughput=-1, + client_id=producer_client_id, version=client_version) producer.run() @@ -144,7 +159,7 @@ def test_quota(self, quota_type, override_quota=True, producer_num=1, consumer_n consumer = ConsoleConsumer(self.test_context, consumer_num, self.kafka, self.topic, consumer_timeout_ms=60000, client_id=consumer_client_id, jmx_object_names=['kafka.consumer:type=consumer-fetch-manager-metrics,client-id=%s' % consumer_client_id], - jmx_attributes=['bytes-consumed-rate']) + jmx_attributes=['bytes-consumed-rate'], version=client_version) consumer.run() for idx, messages in consumer.messages_consumed.iteritems(): diff --git a/tests/kafkatest/tests/connect/connect_distributed_test.py b/tests/kafkatest/tests/connect/connect_distributed_test.py index da7d1deedb915..a27b54d6f89ee 100644 --- a/tests/kafkatest/tests/connect/connect_distributed_test.py +++ b/tests/kafkatest/tests/connect/connect_distributed_test.py @@ -339,7 +339,7 @@ def test_file_source_and_sink(self, security_protocol): node.account.ssh("echo -e -n " + repr(self.SECOND_INPUTS) + " >> " + self.INPUT_FILE) wait_until(lambda: self._validate_file_output(self.FIRST_INPUT_LIST + self.SECOND_INPUT_LIST), timeout_sec=70, err_msg="Sink output file never converged to the same state as the input file") - @cluster(num_nodes=5) + @cluster(num_nodes=6) @matrix(clean=[True, False]) def test_bounce(self, clean): """ diff --git a/tests/kafkatest/tests/connect/connect_rest_test.py b/tests/kafkatest/tests/connect/connect_rest_test.py index 3c7cd899f069a..ce374182b619d 100644 --- a/tests/kafkatest/tests/connect/connect_rest_test.py +++ b/tests/kafkatest/tests/connect/connect_rest_test.py @@ -14,7 +14,7 @@ # limitations under the License. from kafkatest.tests.kafka_test import KafkaTest -from kafkatest.services.connect import ConnectDistributedService, ConnectRestError +from kafkatest.services.connect import ConnectDistributedService, ConnectRestError, ConnectServiceBase from ducktape.utils.util import wait_until from ducktape.mark.resource import cluster from ducktape.cluster.remoteaccount import RemoteCommandError @@ -31,14 +31,21 @@ class ConnectRestApiTest(KafkaTest): FILE_SOURCE_CONNECTOR = 'org.apache.kafka.connect.file.FileStreamSourceConnector' FILE_SINK_CONNECTOR = 'org.apache.kafka.connect.file.FileStreamSinkConnector' - FILE_SOURCE_CONFIGS = {'name', 'connector.class', 'tasks.max', 'key.converter', 'value.converter', 'header.converter', 'batch.size', 'topic', 'file', 'transforms'} - FILE_SINK_CONFIGS = {'name', 'connector.class', 'tasks.max', 'key.converter', 'value.converter', 'header.converter', 'topics', 'file', 'transforms', 'topics.regex'} + FILE_SOURCE_CONFIGS = {'name', 'connector.class', 'tasks.max', 'key.converter', 'value.converter', 'header.converter', 'batch.size', + 'topic', 'file', 'transforms', 'config.action.reload', 'errors.retry.timeout', 'errors.retry.delay.max.ms', + 'errors.tolerance', 'errors.log.enable', 'errors.log.include.messages'} + FILE_SINK_CONFIGS = {'name', 'connector.class', 'tasks.max', 'key.converter', 'value.converter', 'header.converter', 'topics', + 'file', 'transforms', 'topics.regex', 'config.action.reload', 'errors.retry.timeout', 'errors.retry.delay.max.ms', + 'errors.tolerance', 'errors.log.enable', 'errors.log.include.messages', 'errors.deadletterqueue.topic.name', + 'errors.deadletterqueue.topic.replication.factor', 'errors.deadletterqueue.context.headers.enable'} INPUT_FILE = "/mnt/connect.input" INPUT_FILE2 = "/mnt/connect.input2" OUTPUT_FILE = "/mnt/connect.output" - TOPIC = "test" + TOPIC = "topic-${file:%s:topic.external}" % ConnectServiceBase.EXTERNAL_CONFIGS_FILE + TOPIC_TEST = "test" + DEFAULT_BATCH_SIZE = "2000" OFFSETS_TOPIC = "connect-offsets" OFFSETS_REPLICATION_FACTOR = "1" @@ -73,6 +80,7 @@ def test_rest_api(self): self.schemas = True self.cc.set_configs(lambda node: self.render("connect-distributed.properties", node=node)) + self.cc.set_external_configs(lambda node: self.render("connect-file-external.properties", node=node)) self.cc.start() diff --git a/tests/kafkatest/tests/connect/connect_test.py b/tests/kafkatest/tests/connect/connect_test.py index 3753876333789..9a1ff1bb63123 100644 --- a/tests/kafkatest/tests/connect/connect_test.py +++ b/tests/kafkatest/tests/connect/connect_test.py @@ -18,15 +18,17 @@ from ducktape.utils.util import wait_until from ducktape.mark import parametrize, matrix from ducktape.cluster.remoteaccount import RemoteCommandError +from ducktape.errors import TimeoutError from kafkatest.services.zookeeper import ZookeeperService from kafkatest.services.kafka import KafkaService -from kafkatest.services.connect import ConnectStandaloneService +from kafkatest.services.connect import ConnectServiceBase, ConnectStandaloneService, ErrorTolerance from kafkatest.services.console_consumer import ConsoleConsumer from kafkatest.services.security.security_config import SecurityConfig import hashlib import json +import os.path class ConnectStandaloneFileTest(Test): @@ -44,7 +46,8 @@ class ConnectStandaloneFileTest(Test): OFFSETS_FILE = "/mnt/connect.offsets" - TOPIC = "test" + TOPIC = "${file:%s:topic.external}" % ConnectServiceBase.EXTERNAL_CONFIGS_FILE + TOPIC_TEST = "test" FIRST_INPUT_LIST = ["foo", "bar", "baz"] FIRST_INPUT = "\n".join(FIRST_INPUT_LIST) + "\n" @@ -90,7 +93,7 @@ def test_file_source_and_sink(self, converter="org.apache.kafka.connect.json.Jso self.source = ConnectStandaloneService(self.test_context, self.kafka, [self.INPUT_FILE, self.OFFSETS_FILE]) self.sink = ConnectStandaloneService(self.test_context, self.kafka, [self.OUTPUT_FILE, self.OFFSETS_FILE]) - self.consumer_validator = ConsoleConsumer(self.test_context, 1, self.kafka, self.TOPIC, + self.consumer_validator = ConsoleConsumer(self.test_context, 1, self.kafka, self.TOPIC_TEST, consumer_timeout_ms=10000) self.zk.start() @@ -99,6 +102,9 @@ def test_file_source_and_sink(self, converter="org.apache.kafka.connect.json.Jso self.source.set_configs(lambda node: self.render("connect-standalone.properties", node=node), [self.render("connect-file-source.properties")]) self.sink.set_configs(lambda node: self.render("connect-standalone.properties", node=node), [self.render("connect-file-sink.properties")]) + self.source.set_external_configs(lambda node: self.render("connect-file-external.properties", node=node)) + self.sink.set_external_configs(lambda node: self.render("connect-file-external.properties", node=node)) + self.source.start() self.sink.start() @@ -127,3 +133,76 @@ def validate_output(self, value): return output_hash == hashlib.md5(value).hexdigest() except RemoteCommandError: return False + + @cluster(num_nodes=5) + @parametrize(error_tolerance=ErrorTolerance.ALL) + @parametrize(error_tolerance=ErrorTolerance.NONE) + def test_skip_and_log_to_dlq(self, error_tolerance): + self.kafka = KafkaService(self.test_context, self.num_brokers, self.zk, topics=self.topics) + + # set config props + self.override_error_tolerance_props = error_tolerance + self.enable_deadletterqueue = True + + successful_records = [] + faulty_records = [] + records = [] + for i in range(0, 1000): + if i % 2 == 0: + records.append('{"some_key":' + str(i) + '}') + successful_records.append('{some_key=' + str(i) + '}') + else: + # badly formatted json records (missing a quote after the key) + records.append('{"some_key:' + str(i) + '}') + faulty_records.append('{"some_key:' + str(i) + '}') + + records = "\n".join(records) + "\n" + successful_records = "\n".join(successful_records) + "\n" + if error_tolerance == ErrorTolerance.ALL: + faulty_records = ",".join(faulty_records) + else: + faulty_records = faulty_records[0] + + self.source = ConnectStandaloneService(self.test_context, self.kafka, [self.INPUT_FILE, self.OFFSETS_FILE]) + self.sink = ConnectStandaloneService(self.test_context, self.kafka, [self.OUTPUT_FILE, self.OFFSETS_FILE]) + + self.zk.start() + self.kafka.start() + + self.override_key_converter = "org.apache.kafka.connect.storage.StringConverter" + self.override_value_converter = "org.apache.kafka.connect.storage.StringConverter" + self.source.set_configs(lambda node: self.render("connect-standalone.properties", node=node), [self.render("connect-file-source.properties")]) + + self.override_key_converter = "org.apache.kafka.connect.json.JsonConverter" + self.override_value_converter = "org.apache.kafka.connect.json.JsonConverter" + self.override_key_converter_schemas_enable = False + self.override_value_converter_schemas_enable = False + self.sink.set_configs(lambda node: self.render("connect-standalone.properties", node=node), [self.render("connect-file-sink.properties")]) + + self.source.set_external_configs(lambda node: self.render("connect-file-external.properties", node=node)) + self.sink.set_external_configs(lambda node: self.render("connect-file-external.properties", node=node)) + + 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(records) + " >> " + self.INPUT_FILE) + + if error_tolerance == ErrorTolerance.NONE: + try: + wait_until(lambda: self.validate_output(successful_records), timeout_sec=15, + err_msg="Clean records added to input file were not seen in the output file in a reasonable amount of time.") + raise Exception("Expected to not find any results in this file.") + except TimeoutError: + self.logger.info("Caught expected exception") + else: + wait_until(lambda: self.validate_output(successful_records), timeout_sec=15, + err_msg="Clean records added to input file were not seen in the output file in a reasonable amount of time.") + + if self.enable_deadletterqueue: + self.logger.info("Reading records from deadletterqueue") + consumer_validator = ConsoleConsumer(self.test_context, 1, self.kafka, "my-connector-errors", + consumer_timeout_ms=10000) + consumer_validator.run() + actual = ",".join(consumer_validator.messages_consumed[1]) + assert faulty_records == actual, "Expected %s but saw %s in dead letter queue" % (faulty_records, actual) diff --git a/tests/kafkatest/tests/connect/templates/connect-distributed.properties b/tests/kafkatest/tests/connect/templates/connect-distributed.properties index 186773e7d1adb..ca8c4f84efb3e 100644 --- a/tests/kafkatest/tests/connect/templates/connect-distributed.properties +++ b/tests/kafkatest/tests/connect/templates/connect-distributed.properties @@ -50,3 +50,9 @@ consumer.session.timeout.ms=10000 # Reduce the admin client request timeouts so that we don't wait the default 120 sec before failing to connect the admin client request.timeout.ms=30000 + +# Allow connector configs to use externalized config values of the form: +# ${file:/mnt/connect/connect-external-configs.properties:topic.external} +# +config.providers=file +config.providers.file.class=org.apache.kafka.common.config.provider.FileConfigProvider diff --git a/bin/kafka-simple-consumer-shell.sh b/tests/kafkatest/tests/connect/templates/connect-file-external.properties old mode 100755 new mode 100644 similarity index 89% rename from bin/kafka-simple-consumer-shell.sh rename to tests/kafkatest/tests/connect/templates/connect-file-external.properties index 27e386ad7ee87..8dccd2571f581 --- a/bin/kafka-simple-consumer-shell.sh +++ b/tests/kafkatest/tests/connect/templates/connect-file-external.properties @@ -1,17 +1,16 @@ -#!/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.tools.SimpleConsumerShell "$@" +topic.external={{ TOPIC_TEST }} diff --git a/tests/kafkatest/tests/connect/templates/connect-file-sink.properties b/tests/kafkatest/tests/connect/templates/connect-file-sink.properties index bff002bbdb074..a58cc6b68c305 100644 --- a/tests/kafkatest/tests/connect/templates/connect-file-sink.properties +++ b/tests/kafkatest/tests/connect/templates/connect-file-sink.properties @@ -25,4 +25,20 @@ key.converter={{ override_key_converter }} {% endif %} {% if override_key_converter is defined %} value.converter={{ override_value_converter }} -{% endif %} \ No newline at end of file +{% endif %} + +key.converter.schemas.enable={{ override_key_converter_schemas_enable|default(True) }} +value.converter.schemas.enable={{ override_value_converter_schemas_enable|default(True) }} + +# log error context along with application logs +errors.log.enable=true +errors.log.include.messages=true + +{% if enable_deadletterqueue is defined %} +# produce error context into the Kafka topic +errors.deadletterqueue.topic.name={{ override_deadletterqueue_topic_name|default("my-connector-errors") }} +errors.deadletterqueue.topic.replication.factor={{ override_deadletterqueue_replication_factor|default(1) }} +{% endif %} + +# Tolerate all errors. +errors.tolerance={{ override_error_tolerance_props|default("none") }} diff --git a/tests/kafkatest/tests/connect/templates/connect-file-source.properties b/tests/kafkatest/tests/connect/templates/connect-file-source.properties index 800d6a0751ba7..147e85a89d525 100644 --- a/tests/kafkatest/tests/connect/templates/connect-file-source.properties +++ b/tests/kafkatest/tests/connect/templates/connect-file-source.properties @@ -26,3 +26,10 @@ key.converter={{ override_key_converter }} {% if override_key_converter is defined %} value.converter={{ override_value_converter }} {% endif %} + +# log error context along with application logs +errors.log.enable=true +errors.log.include.messages=true + +# Tolerate all errors. +errors.tolerance={{ override_error_tolerance_props|default("none") }} diff --git a/tests/kafkatest/tests/connect/templates/connect-standalone.properties b/tests/kafkatest/tests/connect/templates/connect-standalone.properties index a8eaa44832eba..a471dd5879c47 100644 --- a/tests/kafkatest/tests/connect/templates/connect-standalone.properties +++ b/tests/kafkatest/tests/connect/templates/connect-standalone.properties @@ -31,3 +31,9 @@ offset.storage.file.filename={{ OFFSETS_FILE }} # Reduce the admin client request timeouts so that we don't wait the default 120 sec before failing to connect the admin client request.timeout.ms=30000 + +# Allow connector configs to use externalized config values of the form: +# ${file:/mnt/connect/connect-external-configs.properties:topic.external} +# +config.providers=file +config.providers.file.class=org.apache.kafka.common.config.provider.FileConfigProvider diff --git a/tests/kafkatest/tests/core/compatibility_test_new_broker_test.py b/tests/kafkatest/tests/core/compatibility_test_new_broker_test.py index 0d6ad7d0e2c52..8e9b30875184f 100644 --- a/tests/kafkatest/tests/core/compatibility_test_new_broker_test.py +++ b/tests/kafkatest/tests/core/compatibility_test_new_broker_test.py @@ -23,7 +23,7 @@ from kafkatest.services.zookeeper import ZookeeperService from kafkatest.tests.produce_consume_validate import ProduceConsumeValidateTest from kafkatest.utils import is_int -from kafkatest.version import LATEST_0_8_2, LATEST_0_9, LATEST_0_10_0, LATEST_0_10_1, LATEST_0_10_2, LATEST_0_11_0, DEV_BRANCH, KafkaVersion +from kafkatest.version import LATEST_0_8_2, LATEST_0_9, LATEST_0_10_0, LATEST_0_10_1, LATEST_0_10_2, LATEST_0_11_0, LATEST_1_0, LATEST_1_1, DEV_BRANCH, KafkaVersion # Compatibility tests for moving to a new broker (e.g., 0.10.x) and using a mix of old and new clients (e.g., 0.9.x) class ClientCompatibilityTestNewBroker(ProduceConsumeValidateTest): @@ -45,14 +45,16 @@ def setUp(self): @cluster(num_nodes=6) @parametrize(producer_version=str(DEV_BRANCH), consumer_version=str(DEV_BRANCH), compression_types=["snappy"], timestamp_type=str("LogAppendTime")) - @parametrize(producer_version=str(DEV_BRANCH), consumer_version=str(DEV_BRANCH), compression_types=["none"], new_consumer=False, timestamp_type=str("LogAppendTime")) + @parametrize(producer_version=str(DEV_BRANCH), consumer_version=str(DEV_BRANCH), compression_types=["none"], timestamp_type=str("LogAppendTime")) @parametrize(producer_version=str(DEV_BRANCH), consumer_version=str(LATEST_0_9), compression_types=["none"], new_consumer=False, timestamp_type=None) @parametrize(producer_version=str(DEV_BRANCH), consumer_version=str(LATEST_0_9), compression_types=["snappy"], timestamp_type=str("CreateTime")) + @parametrize(producer_version=str(LATEST_1_1), consumer_version=str(LATEST_1_1), compression_types=["lz4"], timestamp_type=str("CreateTime")) + @parametrize(producer_version=str(LATEST_1_0), consumer_version=str(LATEST_1_0), compression_types=["none"], timestamp_type=str("CreateTime")) @parametrize(producer_version=str(LATEST_0_11_0), consumer_version=str(LATEST_0_11_0), compression_types=["gzip"], timestamp_type=str("CreateTime")) @parametrize(producer_version=str(LATEST_0_10_2), consumer_version=str(LATEST_0_10_2), compression_types=["lz4"], timestamp_type=str("CreateTime")) @parametrize(producer_version=str(LATEST_0_10_1), consumer_version=str(LATEST_0_10_1), compression_types=["snappy"], timestamp_type=str("LogAppendTime")) @parametrize(producer_version=str(LATEST_0_10_0), consumer_version=str(LATEST_0_10_0), compression_types=["snappy"], timestamp_type=str("LogAppendTime")) - @parametrize(producer_version=str(LATEST_0_9), consumer_version=str(DEV_BRANCH), compression_types=["none"], new_consumer=False, timestamp_type=None) + @parametrize(producer_version=str(LATEST_0_9), consumer_version=str(DEV_BRANCH), compression_types=["none"], timestamp_type=None) @parametrize(producer_version=str(LATEST_0_9), consumer_version=str(DEV_BRANCH), compression_types=["snappy"], timestamp_type=None) @parametrize(producer_version=str(LATEST_0_9), consumer_version=str(LATEST_0_9), compression_types=["snappy"], timestamp_type=str("LogAppendTime")) @parametrize(producer_version=str(LATEST_0_8_2), consumer_version=str(LATEST_0_8_2), compression_types=["none"], new_consumer=False, timestamp_type=None) diff --git a/tests/kafkatest/tests/core/consumer_group_command_test.py b/tests/kafkatest/tests/core/consumer_group_command_test.py index c03022a630544..4976f3b93fc80 100644 --- a/tests/kafkatest/tests/core/consumer_group_command_test.py +++ b/tests/kafkatest/tests/core/consumer_group_command_test.py @@ -57,15 +57,14 @@ def start_kafka(self, security_protocol, interbroker_security_protocol): interbroker_security_protocol=interbroker_security_protocol, topics=self.topics) self.kafka.start() - def start_consumer(self, security_protocol): - enable_new_consumer = security_protocol == SecurityConfig.SSL + def start_consumer(self): self.consumer = ConsoleConsumer(self.test_context, num_nodes=self.num_brokers, kafka=self.kafka, topic=TOPIC, - consumer_timeout_ms=None, new_consumer=enable_new_consumer) + consumer_timeout_ms=None) self.consumer.start() def setup_and_verify(self, security_protocol, group=None): self.start_kafka(security_protocol, security_protocol) - self.start_consumer(security_protocol) + self.start_consumer() consumer_node = self.consumer.nodes[0] wait_until(lambda: self.consumer.alive(consumer_node), timeout_sec=10, backoff_sec=.2, err_msg="Consumer was too slow to start") @@ -77,16 +76,13 @@ def setup_and_verify(self, security_protocol, group=None): kafka_node.account.create_file(self.COMMAND_CONFIG_FILE, prop_file) # Verify ConsumerGroupCommand lists expected consumer groups - enable_new_consumer = security_protocol != SecurityConfig.PLAINTEXT - command_config_file = None - if enable_new_consumer: - command_config_file = self.COMMAND_CONFIG_FILE + command_config_file = self.COMMAND_CONFIG_FILE if group: - wait_until(lambda: re.search("topic-consumer-group-command",self.kafka.describe_consumer_group(group=group, node=kafka_node, new_consumer=enable_new_consumer, command_config=command_config_file)), timeout_sec=10, + wait_until(lambda: re.search("topic-consumer-group-command",self.kafka.describe_consumer_group(group=group, node=kafka_node, command_config=command_config_file)), timeout_sec=10, err_msg="Timed out waiting to list expected consumer groups.") else: - wait_until(lambda: "test-consumer-group" in self.kafka.list_consumer_groups(node=kafka_node, new_consumer=enable_new_consumer, command_config=command_config_file), timeout_sec=10, + wait_until(lambda: "test-consumer-group" in self.kafka.list_consumer_groups(node=kafka_node, command_config=command_config_file), timeout_sec=10, err_msg="Timed out waiting to list expected consumer groups.") self.consumer.stop() diff --git a/tests/kafkatest/tests/core/get_offset_shell_test.py b/tests/kafkatest/tests/core/get_offset_shell_test.py index e45365d32cf31..b0f67ace15f96 100644 --- a/tests/kafkatest/tests/core/get_offset_shell_test.py +++ b/tests/kafkatest/tests/core/get_offset_shell_test.py @@ -64,10 +64,9 @@ def start_producer(self): wait_until(lambda: self.producer.num_acked >= current_acked + MAX_MESSAGES, timeout_sec=10, err_msg="Timeout awaiting messages to be produced and acked") - def start_consumer(self, security_protocol): - enable_new_consumer = security_protocol != SecurityConfig.PLAINTEXT + def start_consumer(self): self.consumer = ConsoleConsumer(self.test_context, num_nodes=self.num_brokers, kafka=self.kafka, topic=TOPIC, - consumer_timeout_ms=1000, new_consumer=enable_new_consumer) + consumer_timeout_ms=1000) self.consumer.start() @cluster(num_nodes=4) @@ -82,7 +81,7 @@ def test_get_offset_shell(self, security_protocol='PLAINTEXT'): # Assert that offset fetched without any consumers consuming is 0 assert self.kafka.get_offset_shell(TOPIC, None, 1000, 1, -1), "%s:%s:%s" % (TOPIC, NUM_PARTITIONS - 1, 0) - self.start_consumer(security_protocol) + self.start_consumer() node = self.consumer.nodes[0] @@ -90,4 +89,4 @@ def test_get_offset_shell(self, security_protocol='PLAINTEXT'): # Assert that offset is correctly indicated by GetOffsetShell tool wait_until(lambda: "%s:%s:%s" % (TOPIC, NUM_PARTITIONS - 1, MAX_MESSAGES) in self.kafka.get_offset_shell(TOPIC, None, 1000, 1, -1), timeout_sec=10, - err_msg="Timed out waiting to reach expected offset.") \ No newline at end of file + err_msg="Timed out waiting to reach expected offset.") diff --git a/tests/kafkatest/tests/core/log_dir_failure_test.py b/tests/kafkatest/tests/core/log_dir_failure_test.py index faa13c049beca..108d003f4c019 100644 --- a/tests/kafkatest/tests/core/log_dir_failure_test.py +++ b/tests/kafkatest/tests/core/log_dir_failure_test.py @@ -23,7 +23,7 @@ from kafkatest.services.console_consumer import ConsoleConsumer from kafkatest.tests.produce_consume_validate import ProduceConsumeValidateTest from kafkatest.utils import is_int - +from kafkatest.utils.remote_account import path_exists def select_node(test, broker_type, topic): """ Discover node of requested type. For leader type, discovers leader for our topic and partition 0 @@ -67,12 +67,13 @@ def __init__(self, test_context): num_nodes=3, zk=self.zk, topics={ - self.topic1: {"partitions": 1, "replication-factor": 3, "configs": {"min.insync.replicas": 2}}, - self.topic2: {"partitions": 1, "replication-factor": 3, "configs": {"min.insync.replicas": 1}} + self.topic1: {"partitions": 1, "replication-factor": 3, "configs": {"min.insync.replicas": 1}}, + self.topic2: {"partitions": 1, "replication-factor": 3, "configs": {"min.insync.replicas": 2}} }, # Set log.roll.ms to 3 seconds so that broker will detect disk error sooner when it creates log segment # Otherwise broker will still be able to read/write the log file even if the log directory is inaccessible. server_prop_overides=[ + [config_property.OFFSETS_TOPIC_NUM_PARTITIONS, "1"], [config_property.LOG_FLUSH_INTERVAL_MESSAGE, "5"], [config_property.REPLICA_HIGHWATERMARK_CHECKPOINT_INTERVAL_MS, "60000"], [config_property.LOG_ROLL_TIME_MS, "3000"] @@ -98,7 +99,6 @@ def test_replication_with_disk_failure(self, bounce_broker, security_protocol, b Setup: 1 zk, 3 kafka nodes, 1 topic with partitions=3, replication-factor=3, and min.insync.replicas=2 and another topic with partitions=3, replication-factor=3, and min.insync.replicas=1 - - Produce messages in the background - Consume messages in the background - Drive broker failures (shutdown, or bounce repeatedly with kill -15 or kill -9) @@ -111,39 +111,46 @@ def test_replication_with_disk_failure(self, bounce_broker, security_protocol, b self.kafka.start() try: - # Initialize producer/consumer for topic1 - self.producer = VerifiableProducer(self.test_context, self.num_producers, self.kafka, self.topic1, + # Initialize producer/consumer for topic2 + self.producer = VerifiableProducer(self.test_context, self.num_producers, self.kafka, self.topic2, throughput=self.producer_throughput) - self.consumer = ConsoleConsumer(self.test_context, self.num_consumers, self.kafka, self.topic1, group_id="test-consumer-group-1", - new_consumer=False, consumer_timeout_ms=60000, message_validator=is_int) + self.consumer = ConsoleConsumer(self.test_context, self.num_consumers, self.kafka, self.topic2, group_id="test-consumer-group-1", + consumer_timeout_ms=60000, message_validator=is_int) self.start_producer_and_consumer() - # Get a replica of the partition of topic1 and make its first log directory offline by changing the log dir's permission. - # We assume that partition of topic1 is created in the first log directory of respective brokers. - broker_node = select_node(self, broker_type, self.topic1) + # Get a replica of the partition of topic2 and make its log directory offline by changing the log dir's permission. + # We assume that partition of topic2 is created in the second log directory of respective brokers. + broker_node = select_node(self, broker_type, self.topic2) broker_idx = self.kafka.idx(broker_node) - assert broker_idx in self.kafka.isr_idx_list(self.topic1), \ - "Broker %d should be in isr set %s" % (broker_idx, str(self.kafka.isr_idx_list(self.topic1))) - - self.logger.debug("Making log dir %s inaccessible" % (KafkaService.DATA_LOG_DIR_1)) - cmd = "chmod a-w %s -R" % (KafkaService.DATA_LOG_DIR_1) + assert broker_idx in self.kafka.isr_idx_list(self.topic2), \ + "Broker %d should be in isr set %s" % (broker_idx, str(self.kafka.isr_idx_list(self.topic2))) + + # Verify that topic1 and the consumer offset topic is in the first log directory and topic2 is in the second log directory + topic_1_partition_0 = KafkaService.DATA_LOG_DIR_1 + "/test_topic_1-0" + topic_2_partition_0 = KafkaService.DATA_LOG_DIR_2 + "/test_topic_2-0" + offset_topic_partition_0 = KafkaService.DATA_LOG_DIR_1 + "/__consumer_offsets-0" + for path in [topic_1_partition_0, topic_2_partition_0, offset_topic_partition_0]: + assert path_exists(broker_node, path), "%s should exist" % path + + self.logger.debug("Making log dir %s inaccessible" % (KafkaService.DATA_LOG_DIR_2)) + cmd = "chmod a-w %s -R" % (KafkaService.DATA_LOG_DIR_2) broker_node.account.ssh(cmd, allow_fail=False) if bounce_broker: self.kafka.restart_node(broker_node, clean_shutdown=True) # Verify the following: - # 1) The broker with offline log directory is not the leader of the partition of topic1 + # 1) The broker with offline log directory is not the leader of the partition of topic2 # 2) The broker with offline log directory is not in the ISR # 3) The broker with offline log directory is still online - # 4) Messages can still be produced and consumed from topic1 - wait_until(lambda: self.kafka.leader(self.topic1, partition=0) != broker_node, + # 4) Messages can still be produced and consumed from topic2 + wait_until(lambda: self.kafka.leader(self.topic2, partition=0) != broker_node, timeout_sec=60, - err_msg="Broker %d should not be leader of topic %s and partition 0" % (broker_idx, self.topic1)) + err_msg="Broker %d should not be leader of topic %s and partition 0" % (broker_idx, self.topic2)) assert self.kafka.alive(broker_node), "Broker %d should be still online" % (broker_idx) - wait_until(lambda: broker_idx not in self.kafka.isr_idx_list(self.topic1), + wait_until(lambda: broker_idx not in self.kafka.isr_idx_list(self.topic2), timeout_sec=60, - err_msg="Broker %d should not be in isr set %s" % (broker_idx, str(self.kafka.isr_idx_list(self.topic1)))) + err_msg="Broker %d should not be in isr set %s" % (broker_idx, str(self.kafka.isr_idx_list(self.topic2)))) self.stop_producer_and_consumer() self.validate() @@ -157,16 +164,17 @@ def test_replication_with_disk_failure(self, bounce_broker, security_protocol, b self.kafka.stop_node(node) # Verify the following: - # 1) The broker with offline directory is the only in-sync broker of the partition of topic2 - # 2) Messages can still be produced and consumed from topic2 - self.producer = VerifiableProducer(self.test_context, self.num_producers, self.kafka, self.topic2, + # 1) The broker with offline directory is the only in-sync broker of the partition of topic1 + # 2) Messages can still be produced and consumed from topic1 + self.producer = VerifiableProducer(self.test_context, self.num_producers, self.kafka, self.topic1, throughput=self.producer_throughput, offline_nodes=offline_nodes) - self.consumer = ConsoleConsumer(self.test_context, self.num_consumers, self.kafka, self.topic2, group_id="test-consumer-group-2", - new_consumer=False, consumer_timeout_ms=60000, message_validator=is_int) + self.consumer = ConsoleConsumer(self.test_context, self.num_consumers, self.kafka, self.topic1, group_id="test-consumer-group-2", + consumer_timeout_ms=90000, message_validator=is_int) + self.consumer_start_timeout_sec = 90 self.start_producer_and_consumer() - assert self.kafka.isr_idx_list(self.topic2) == [broker_idx], \ - "In-sync replicas of topic %s and partition 0 should be %s" % (self.topic2, str([broker_idx])) + assert self.kafka.isr_idx_list(self.topic1) == [broker_idx], \ + "In-sync replicas of topic %s and partition 0 should be %s" % (self.topic1, str([broker_idx])) self.stop_producer_and_consumer() self.validate() diff --git a/tests/kafkatest/tests/core/mirror_maker_test.py b/tests/kafkatest/tests/core/mirror_maker_test.py index ce86a60cc1135..c33f103e50e9e 100644 --- a/tests/kafkatest/tests/core/mirror_maker_test.py +++ b/tests/kafkatest/tests/core/mirror_maker_test.py @@ -103,7 +103,6 @@ def bounce(self, clean_shutdown=True): self.mirror_maker.start_node(node) # Ensure new messages are once again showing up on the target cluster - # new consumer requires higher timeout here wait_until(lambda: len(self.consumer.messages_consumed[1]) > num_consumed + 100, timeout_sec=60) def wait_for_n_messages(self, n_messages=100): @@ -112,11 +111,10 @@ def wait_for_n_messages(self, n_messages=100): err_msg="Producer failed to produce %d messages in a reasonable amount of time." % n_messages) @cluster(num_nodes=7) - @parametrize(security_protocol='PLAINTEXT', new_consumer=False) - @matrix(security_protocol=['PLAINTEXT', 'SSL'], new_consumer=[True]) + @matrix(security_protocol=['PLAINTEXT', 'SSL']) @cluster(num_nodes=8) - @matrix(security_protocol=['SASL_PLAINTEXT', 'SASL_SSL'], new_consumer=[True]) - def test_simple_end_to_end(self, security_protocol, new_consumer): + @matrix(security_protocol=['SASL_PLAINTEXT', 'SASL_SSL']) + def test_simple_end_to_end(self, security_protocol): """ Test end-to-end behavior under non-failure conditions. @@ -129,27 +127,19 @@ def test_simple_end_to_end(self, security_protocol, new_consumer): - Verify that number of consumed messages matches the number produced. """ self.start_kafka(security_protocol) - self.consumer.new_consumer = new_consumer - - self.mirror_maker.new_consumer = new_consumer self.mirror_maker.start() mm_node = self.mirror_maker.nodes[0] with mm_node.account.monitor_log(self.mirror_maker.LOG_FILE) as monitor: - if new_consumer: - monitor.wait_until("Resetting offset for partition", timeout_sec=30, err_msg="Mirrormaker did not reset fetch offset in a reasonable amount of time.") - else: - monitor.wait_until("reset fetch offset", timeout_sec=30, err_msg="Mirrormaker did not reset fetch offset in a reasonable amount of time.") - + monitor.wait_until("Resetting offset for partition", timeout_sec=30, err_msg="Mirrormaker did not reset fetch offset in a reasonable amount of time.") self.run_produce_consume_validate(core_test_action=self.wait_for_n_messages) self.mirror_maker.stop() @cluster(num_nodes=7) - @matrix(offsets_storage=["kafka", "zookeeper"], new_consumer=[False], clean_shutdown=[True, False]) - @matrix(new_consumer=[True], clean_shutdown=[True, False], security_protocol=['PLAINTEXT', 'SSL']) + @matrix(clean_shutdown=[True, False], security_protocol=['PLAINTEXT', 'SSL']) @cluster(num_nodes=8) - @matrix(new_consumer=[True], clean_shutdown=[True, False], security_protocol=['SASL_PLAINTEXT', 'SASL_SSL']) - def test_bounce(self, offsets_storage="kafka", new_consumer=True, clean_shutdown=True, security_protocol='PLAINTEXT'): + @matrix(clean_shutdown=[True, False], security_protocol=['SASL_PLAINTEXT', 'SASL_SSL']) + def test_bounce(self, offsets_storage="kafka", clean_shutdown=True, security_protocol='PLAINTEXT'): """ Test end-to-end behavior under failure conditions. @@ -161,26 +151,21 @@ def test_bounce(self, offsets_storage="kafka", new_consumer=True, clean_shutdown - Bounce MM process - Verify every message acknowledged by the source producer is consumed by the target consumer """ - if new_consumer and not clean_shutdown: - # Increase timeout on downstream console consumer; mirror maker with new consumer takes extra time + if not clean_shutdown: + # Increase timeout on downstream console consumer; mirror maker takes extra time # during hard bounce. This is because the restarted mirror maker consumer won't be able to rejoin # the group until the previous session times out self.consumer.consumer_timeout_ms = 60000 self.start_kafka(security_protocol) - self.consumer.new_consumer = new_consumer self.mirror_maker.offsets_storage = offsets_storage - self.mirror_maker.new_consumer = new_consumer self.mirror_maker.start() # Wait until mirror maker has reset fetch offset at least once before continuing with the rest of the test mm_node = self.mirror_maker.nodes[0] with mm_node.account.monitor_log(self.mirror_maker.LOG_FILE) as monitor: - if new_consumer: - monitor.wait_until("Resetting offset for partition", timeout_sec=30, err_msg="Mirrormaker did not reset fetch offset in a reasonable amount of time.") - else: - monitor.wait_until("reset fetch offset", timeout_sec=30, err_msg="Mirrormaker did not reset fetch offset in a reasonable amount of time.") + monitor.wait_until("Resetting offset for partition", timeout_sec=30, err_msg="Mirrormaker did not reset fetch offset in a reasonable amount of time.") self.run_produce_consume_validate(core_test_action=lambda: self.bounce(clean_shutdown=clean_shutdown)) self.mirror_maker.stop() diff --git a/tests/kafkatest/tests/core/replication_test.py b/tests/kafkatest/tests/core/replication_test.py index 30fad4e38cc41..f5c64222bdb65 100644 --- a/tests/kafkatest/tests/core/replication_test.py +++ b/tests/kafkatest/tests/core/replication_test.py @@ -19,12 +19,7 @@ from ducktape.mark import parametrize from ducktape.mark.resource import cluster -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 -from kafkatest.tests.produce_consume_validate import ProduceConsumeValidateTest -from kafkatest.utils import is_int +from kafkatest.tests.end_to_end import EndToEndTest import signal @@ -83,7 +78,7 @@ def hard_bounce(test, broker_type): } -class ReplicationTest(ProduceConsumeValidateTest): +class ReplicationTest(EndToEndTest): """ 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 @@ -98,25 +93,16 @@ class ReplicationTest(ProduceConsumeValidateTest): indicator that nothing is left to consume. """ + TOPIC_CONFIG = { + "partitions": 3, + "replication-factor": 3, + "configs": {"min.insync.replicas": 2} + } + 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.kafka = KafkaService(test_context, num_nodes=3, zk=self.zk, - topics={self.topic: { - "partitions": 3, - "replication-factor": 3, - 'configs': {"min.insync.replicas": 2}} - }) - self.producer_throughput = 1000 - self.num_producers = 1 - self.num_consumers = 1 - - def setUp(self): - self.zk.start() - + super(ReplicationTest, self).__init__(test_context=test_context, topic_config=self.TOPIC_CONFIG) + 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 @@ -156,16 +142,23 @@ def test_replication_with_broker_failure(self, failure_mode, security_protocol, - Validate that every acked message was consumed """ - self.kafka.security_protocol = security_protocol - self.kafka.interbroker_security_protocol = security_protocol - self.kafka.client_sasl_mechanism = client_sasl_mechanism - self.kafka.interbroker_sasl_mechanism = interbroker_sasl_mechanism - new_consumer = False if self.kafka.security_protocol == "PLAINTEXT" else True - self.enable_idempotence = enable_idempotence - compression_types = None if not compression_type else [compression_type] * self.num_producers - self.producer = VerifiableProducer(self.test_context, self.num_producers, self.kafka, self.topic, - throughput=self.producer_throughput, compression_types=compression_types, - enable_idempotence=enable_idempotence) - self.consumer = ConsoleConsumer(self.test_context, self.num_consumers, self.kafka, self.topic, new_consumer=new_consumer, consumer_timeout_ms=60000, message_validator=is_int) + self.create_zookeeper() + self.zk.start() + + self.create_kafka(num_nodes=3, + security_protocol=security_protocol, + interbroker_security_protocol=security_protocol, + client_sasl_mechanism=client_sasl_mechanism, + interbroker_sasl_mechanism=interbroker_sasl_mechanism) self.kafka.start() - self.run_produce_consume_validate(core_test_action=lambda: failures[failure_mode](self, broker_type)) + + compression_types = None if not compression_type else [compression_type] + self.create_producer(compression_types=compression_types, enable_idempotence=enable_idempotence) + self.producer.start() + + self.create_consumer(log_level="DEBUG") + self.consumer.start() + + self.await_startup() + failures[failure_mode](self, broker_type) + self.run_validation(enable_idempotence=enable_idempotence) diff --git a/tests/kafkatest/tests/core/security_test.py b/tests/kafkatest/tests/core/security_test.py index 4edbcff893768..d62735a549bb2 100644 --- a/tests/kafkatest/tests/core/security_test.py +++ b/tests/kafkatest/tests/core/security_test.py @@ -19,14 +19,9 @@ from ducktape.utils.util import wait_until from ducktape.errors import TimeoutError -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 from kafkatest.services.security.security_config import SecurityConfig from kafkatest.services.security.security_config import SslStores -from kafkatest.tests.produce_consume_validate import ProduceConsumeValidateTest -from kafkatest.utils import is_int +from kafkatest.tests.end_to_end import EndToEndTest class TestSslStores(SslStores): def __init__(self, local_scratch_dir, valid_hostname=True): @@ -41,7 +36,7 @@ def hostname(self, node): else: return "invalidhostname" -class SecurityTest(ProduceConsumeValidateTest): +class SecurityTest(EndToEndTest): """ These tests validate security features. """ @@ -50,21 +45,6 @@ def __init__(self, test_context): """:type test_context: ducktape.tests.test.TestContext""" super(SecurityTest, self).__init__(test_context=test_context) - self.topic = "test_topic" - self.zk = ZookeeperService(test_context, num_nodes=1) - self.kafka = KafkaService(test_context, num_nodes=1, zk=self.zk, topics={self.topic: { - "partitions": 2, - "replication-factor": 1} - }) - self.num_partitions = 2 - self.timeout_sec = 10000 - self.producer_throughput = 1000 - self.num_producers = 1 - self.num_consumers = 1 - - def setUp(self): - self.zk.start() - def producer_consumer_have_expected_error(self, error): try: for node in self.producer.nodes: @@ -87,16 +67,19 @@ def test_client_ssl_endpoint_validation_failure(self, security_protocol, interbr with hostname verification failure. Hence clients are expected to fail with LEADER_NOT_AVAILABLE. """ - self.kafka.security_protocol = security_protocol - self.kafka.interbroker_security_protocol = interbroker_security_protocol - SecurityConfig.ssl_stores = TestSslStores(self.test_context.local_scratch_dir, valid_hostname=False) + SecurityConfig.ssl_stores = TestSslStores(self.test_context.local_scratch_dir, + valid_hostname=False) + + self.create_zookeeper() + self.zk.start() + self.create_kafka(security_protocol=security_protocol, + interbroker_security_protocol=interbroker_security_protocol) self.kafka.start() - self.create_producer_and_consumer() - self.producer.log_level = "TRACE" - self.producer.start() - self.consumer.start() + # We need more verbose logging to catch the expected errors + self.create_and_start_clients(log_level="DEBUG") + try: wait_until(lambda: self.producer.num_acked > 0, timeout_sec=5) @@ -108,20 +91,18 @@ def test_client_ssl_endpoint_validation_failure(self, security_protocol, interbr pass error = 'SSLHandshakeException' if security_protocol == 'SSL' else 'LEADER_NOT_AVAILABLE' - wait_until(lambda: self.producer_consumer_have_expected_error(error), timeout_sec=5) - + wait_until(lambda: self.producer_consumer_have_expected_error(error), timeout_sec=30) self.producer.stop() self.consumer.stop() - self.producer.log_level = "INFO" SecurityConfig.ssl_stores.valid_hostname = True - for node in self.kafka.nodes: - self.kafka.restart_node(node, clean_shutdown=True) - - self.create_producer_and_consumer() - self.run_produce_consume_validate() + self.kafka.restart_cluster() + self.create_and_start_clients(log_level="INFO") + self.run_validation() - def create_producer_and_consumer(self): - self.producer = VerifiableProducer(self.test_context, self.num_producers, self.kafka, self.topic, throughput=self.producer_throughput) - self.consumer = ConsoleConsumer(self.test_context, self.num_consumers, self.kafka, self.topic, consumer_timeout_ms=10000, message_validator=is_int) + def create_and_start_clients(self, log_level): + self.create_producer(log_level=log_level) + self.producer.start() + self.create_consumer(log_level=log_level) + self.consumer.start() diff --git a/tests/kafkatest/tests/core/simple_consumer_shell_test.py b/tests/kafkatest/tests/core/simple_consumer_shell_test.py deleted file mode 100644 index 882aae7fba63d..0000000000000 --- a/tests/kafkatest/tests/core/simple_consumer_shell_test.py +++ /dev/null @@ -1,79 +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. - - -from ducktape.utils.util import wait_until -from ducktape.tests.test import Test -from ducktape.mark.resource import cluster - -from kafkatest.services.simple_consumer_shell import SimpleConsumerShell -from kafkatest.services.verifiable_producer import VerifiableProducer - -from kafkatest.services.zookeeper import ZookeeperService -from kafkatest.services.kafka import KafkaService -TOPIC = "topic-simple-consumer-shell" -MAX_MESSAGES = 100 -NUM_PARTITIONS = 1 -REPLICATION_FACTOR = 1 - - -class SimpleConsumerShellTest(Test): - """ - Tests SimpleConsumerShell tool - """ - def __init__(self, test_context): - super(SimpleConsumerShellTest, self).__init__(test_context) - self.num_zk = 1 - self.num_brokers = 1 - self.messages_received_count = 0 - self.topics = { - TOPIC: {'partitions': NUM_PARTITIONS, 'replication-factor': REPLICATION_FACTOR} - } - - self.zk = ZookeeperService(test_context, self.num_zk) - - def setUp(self): - self.zk.start() - - def start_kafka(self): - self.kafka = KafkaService( - self.test_context, self.num_brokers, - self.zk, topics=self.topics) - self.kafka.start() - - def run_producer(self): - # This will produce to kafka cluster - self.producer = VerifiableProducer(self.test_context, num_nodes=1, kafka=self.kafka, topic=TOPIC, throughput=1000, max_messages=MAX_MESSAGES) - self.producer.start() - wait_until(lambda: self.producer.num_acked == MAX_MESSAGES, timeout_sec=10, - err_msg="Timeout awaiting messages to be produced and acked") - - def start_simple_consumer_shell(self): - self.simple_consumer_shell = SimpleConsumerShell(self.test_context, 1, self.kafka, TOPIC) - self.simple_consumer_shell.start() - - @cluster(num_nodes=4) - def test_simple_consumer_shell(self): - """ - Tests if SimpleConsumerShell is fetching expected records - :return: None - """ - self.start_kafka() - self.run_producer() - self.start_simple_consumer_shell() - - # Assert that SimpleConsumerShell is fetching expected number of messages - wait_until(lambda: self.simple_consumer_shell.get_output().count("\n") == (MAX_MESSAGES + 1), timeout_sec=10, - err_msg="Timed out waiting to receive expected number of messages.") \ No newline at end of file diff --git a/tests/kafkatest/tests/core/transactions_test.py b/tests/kafkatest/tests/core/transactions_test.py index 0914844ecd521..4da5960983c04 100644 --- a/tests/kafkatest/tests/core/transactions_test.py +++ b/tests/kafkatest/tests/core/transactions_test.py @@ -136,7 +136,6 @@ def start_consumer(self, topic_to_read, group_id): kafka=self.kafka, topic=topic_to_read, group_id=group_id, - new_consumer=True, message_validator=is_int, from_beginning=True, isolation_level="read_committed") diff --git a/tests/kafkatest/tests/core/upgrade_test.py b/tests/kafkatest/tests/core/upgrade_test.py index c8cdac7e4ce8a..bfbb5b3a6073e 100644 --- a/tests/kafkatest/tests/core/upgrade_test.py +++ b/tests/kafkatest/tests/core/upgrade_test.py @@ -23,7 +23,7 @@ from kafkatest.services.zookeeper import ZookeeperService from kafkatest.tests.produce_consume_validate import ProduceConsumeValidateTest from kafkatest.utils import is_int -from kafkatest.version import LATEST_0_8_2, LATEST_0_9, LATEST_0_10, LATEST_0_10_0, LATEST_0_10_1, LATEST_0_10_2, LATEST_0_11_0, DEV_BRANCH, KafkaVersion +from kafkatest.version import LATEST_0_8_2, LATEST_0_9, LATEST_0_10, LATEST_0_10_0, LATEST_0_10_1, LATEST_0_10_2, LATEST_0_11_0, LATEST_1_0, LATEST_1_1, V_0_9_0_0, DEV_BRANCH, KafkaVersion class TestUpgrade(ProduceConsumeValidateTest): @@ -36,7 +36,7 @@ def setUp(self): self.zk.start() # Producer and consumer - self.producer_throughput = 10000 + self.producer_throughput = 1000 self.num_producers = 1 self.num_consumers = 1 @@ -60,33 +60,33 @@ def perform_upgrade(self, from_kafka_version, to_message_format_version=None): self.kafka.start_node(node) @cluster(num_nodes=6) - @parametrize(from_kafka_version=str(LATEST_0_11_0), to_message_format_version=None, compression_types=["gzip"], new_consumer=False) + @parametrize(from_kafka_version=str(LATEST_1_1), to_message_format_version=None, compression_types=["none"]) + @parametrize(from_kafka_version=str(LATEST_1_1), to_message_format_version=None, compression_types=["lz4"]) + @parametrize(from_kafka_version=str(LATEST_1_0), to_message_format_version=None, compression_types=["none"]) + @parametrize(from_kafka_version=str(LATEST_1_0), to_message_format_version=None, compression_types=["snappy"]) + @parametrize(from_kafka_version=str(LATEST_0_11_0), to_message_format_version=None, compression_types=["gzip"]) @parametrize(from_kafka_version=str(LATEST_0_11_0), to_message_format_version=None, compression_types=["lz4"]) @parametrize(from_kafka_version=str(LATEST_0_10_2), to_message_format_version=str(LATEST_0_9), compression_types=["none"]) - @parametrize(from_kafka_version=str(LATEST_0_10_2), to_message_format_version=str(LATEST_0_10), compression_types=["snappy"], new_consumer=False) - @parametrize(from_kafka_version=str(LATEST_0_10_2), to_message_format_version=None, compression_types=["lz4"]) + @parametrize(from_kafka_version=str(LATEST_0_10_2), to_message_format_version=str(LATEST_0_10), compression_types=["snappy"]) @parametrize(from_kafka_version=str(LATEST_0_10_2), to_message_format_version=None, compression_types=["none"]) - @parametrize(from_kafka_version=str(LATEST_0_10_2), to_message_format_version=None, compression_types=["snappy"]) - @parametrize(from_kafka_version=str(LATEST_0_10_2), to_message_format_version=None, compression_types=["lz4"], new_consumer=False) + @parametrize(from_kafka_version=str(LATEST_0_10_2), to_message_format_version=None, compression_types=["lz4"]) @parametrize(from_kafka_version=str(LATEST_0_10_1), to_message_format_version=None, compression_types=["lz4"]) - @parametrize(from_kafka_version=str(LATEST_0_10_1), to_message_format_version=None, compression_types=["snappy"], new_consumer=False) - @parametrize(from_kafka_version=str(LATEST_0_10_0), to_message_format_version=None, compression_types=["snappy"], new_consumer=False) + @parametrize(from_kafka_version=str(LATEST_0_10_1), to_message_format_version=None, compression_types=["snappy"]) + @parametrize(from_kafka_version=str(LATEST_0_10_0), to_message_format_version=None, compression_types=["snappy"]) @parametrize(from_kafka_version=str(LATEST_0_10_0), to_message_format_version=None, compression_types=["lz4"]) @cluster(num_nodes=7) @parametrize(from_kafka_version=str(LATEST_0_9), to_message_format_version=None, compression_types=["none"], security_protocol="SASL_SSL") @cluster(num_nodes=6) @parametrize(from_kafka_version=str(LATEST_0_9), to_message_format_version=None, compression_types=["snappy"]) - @parametrize(from_kafka_version=str(LATEST_0_9), to_message_format_version=None, compression_types=["lz4"], new_consumer=False) @parametrize(from_kafka_version=str(LATEST_0_9), to_message_format_version=None, compression_types=["lz4"]) - @parametrize(from_kafka_version=str(LATEST_0_9), to_message_format_version=str(LATEST_0_9), compression_types=["none"], new_consumer=False) - @parametrize(from_kafka_version=str(LATEST_0_9), to_message_format_version=str(LATEST_0_9), compression_types=["snappy"]) + @parametrize(from_kafka_version=str(LATEST_0_9), to_message_format_version=str(LATEST_0_9), compression_types=["none"]) @parametrize(from_kafka_version=str(LATEST_0_9), to_message_format_version=str(LATEST_0_9), compression_types=["lz4"]) @cluster(num_nodes=7) - @parametrize(from_kafka_version=str(LATEST_0_8_2), to_message_format_version=None, compression_types=["none"], new_consumer=False) - @parametrize(from_kafka_version=str(LATEST_0_8_2), to_message_format_version=None, compression_types=["snappy"], new_consumer=False) + @parametrize(from_kafka_version=str(LATEST_0_8_2), to_message_format_version=None, compression_types=["none"]) + @parametrize(from_kafka_version=str(LATEST_0_8_2), to_message_format_version=None, compression_types=["snappy"]) def test_upgrade(self, from_kafka_version, to_message_format_version, compression_types, - new_consumer=True, security_protocol="PLAINTEXT"): - """Test upgrade of Kafka broker cluster from 0.8.2, 0.9.0, 0.10.0, 0.10.1, 0.10.2 to the current version + security_protocol="PLAINTEXT"): + """Test upgrade of Kafka broker cluster from various versions to the current version from_kafka_version is a Kafka version to upgrade from @@ -121,9 +121,10 @@ def test_upgrade(self, from_kafka_version, to_message_format_version, compressio if from_kafka_version <= LATEST_0_10_0: assert self.kafka.cluster_id() is None + new_consumer = from_kafka_version >= V_0_9_0_0 # TODO - reduce the timeout self.consumer = ConsoleConsumer(self.test_context, self.num_consumers, self.kafka, - self.topic, consumer_timeout_ms=30000, new_consumer=new_consumer, + self.topic, new_consumer=new_consumer, consumer_timeout_ms=30000, message_validator=is_int, version=KafkaVersion(from_kafka_version)) self.run_produce_consume_validate(core_test_action=lambda: self.perform_upgrade(from_kafka_version, diff --git a/tests/kafkatest/tests/end_to_end.py b/tests/kafkatest/tests/end_to_end.py new file mode 100644 index 0000000000000..9cc6b41de2eee --- /dev/null +++ b/tests/kafkatest/tests/end_to_end.py @@ -0,0 +1,151 @@ +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +from ducktape.tests.test import Test +from ducktape.utils.util import wait_until + +from kafkatest.services.kafka import KafkaService +from kafkatest.services.kafka import TopicPartition +from kafkatest.services.verifiable_producer import VerifiableProducer +from kafkatest.services.verifiable_consumer import VerifiableConsumer +from kafkatest.services.zookeeper import ZookeeperService +from kafkatest.utils import validate_delivery + +import time + +class EndToEndTest(Test): + """This class provides a shared template for tests which follow the common pattern of: + + - produce to a topic in the background + - consume from that topic in the background + - run some logic, e.g. fail topic leader etc. + - perform validation + """ + + DEFAULT_TOPIC_CONFIG = {"partitions": 2, "replication-factor": 1} + + def __init__(self, test_context, topic="test_topic", topic_config=DEFAULT_TOPIC_CONFIG): + super(EndToEndTest, self).__init__(test_context=test_context) + self.topic = topic + self.topic_config = topic_config + self.records_consumed = [] + self.last_consumed_offsets = {} + + def create_zookeeper(self, num_nodes=1, **kwargs): + self.zk = ZookeeperService(self.test_context, num_nodes=num_nodes, **kwargs) + + def create_kafka(self, num_nodes=1, **kwargs): + group_metadata_config = { + "partitions": num_nodes, + "replication-factor": min(num_nodes, 3), + "configs": {"cleanup.policy": "compact"} + } + + topics = { + self.topic: self.topic_config, + "__consumer_offsets": group_metadata_config + } + self.kafka = KafkaService(self.test_context, num_nodes=num_nodes, + zk=self.zk, topics=topics, **kwargs) + + def create_consumer(self, num_nodes=1, group_id="test_group", **kwargs): + self.consumer = VerifiableConsumer(self.test_context, + num_nodes=num_nodes, + kafka=self.kafka, + topic=self.topic, + group_id=group_id, + on_record_consumed=self.on_record_consumed, + **kwargs) + + + def create_producer(self, num_nodes=1, throughput=1000, **kwargs): + self.producer = VerifiableProducer(self.test_context, + num_nodes=num_nodes, + kafka=self.kafka, + topic=self.topic, + throughput=throughput, + **kwargs) + + def on_record_consumed(self, record, node): + partition = TopicPartition(record["topic"], record["partition"]) + record_id = int(record["value"]) + offset = record["offset"] + self.last_consumed_offsets[partition] = offset + self.records_consumed.append(record_id) + + def await_consumed_offsets(self, last_acked_offsets, timeout_sec): + def has_finished_consuming(): + for partition, offset in last_acked_offsets.iteritems(): + if not partition in self.last_consumed_offsets: + return False + if self.last_consumed_offsets[partition] < offset: + return False + return True + + wait_until(has_finished_consuming, + timeout_sec=timeout_sec, + err_msg="Consumer failed to consume up to offsets %s after waiting %ds." %\ + (str(last_acked_offsets), timeout_sec)) + + + def _collect_all_logs(self): + for s in self.test_context.services: + self.mark_for_collect(s) + + def await_startup(self, min_records=5, timeout_sec=30): + try: + wait_until(lambda: self.consumer.total_consumed() >= min_records, + timeout_sec=timeout_sec, + err_msg="Timed out after %ds while awaiting initial record delivery of %d records" %\ + (timeout_sec, min_records)) + except BaseException: + self._collect_all_logs() + raise + + def run_validation(self, min_records=5000, producer_timeout_sec=30, + consumer_timeout_sec=30, enable_idempotence=False): + try: + wait_until(lambda: self.producer.num_acked > min_records, + timeout_sec=producer_timeout_sec, + err_msg="Producer failed to produce messages for %ds." %\ + producer_timeout_sec) + + self.logger.info("Stopping producer after writing up to offsets %s" %\ + str(self.producer.last_acked_offsets)) + self.producer.stop() + + self.await_consumed_offsets(self.producer.last_acked_offsets, consumer_timeout_sec) + self.consumer.stop() + + self.validate(enable_idempotence) + except BaseException: + self._collect_all_logs() + raise + + def validate(self, enable_idempotence): + self.logger.info("Number of acked records: %d" % len(self.producer.acked)) + self.logger.info("Number of consumed records: %d" % len(self.records_consumed)) + + def check_lost_data(missing_records): + return self.kafka.search_data_files(self.topic, missing_records) + + succeeded, error_msg = validate_delivery(self.producer.acked, self.records_consumed, + enable_idempotence, check_lost_data) + + # Collect all logs if validation fails + if not succeeded: + self._collect_all_logs() + + assert succeeded, error_msg diff --git a/tests/kafkatest/tests/produce_consume_validate.py b/tests/kafkatest/tests/produce_consume_validate.py index 7a78da3c5f09c..e49d02e6bbf7b 100644 --- a/tests/kafkatest/tests/produce_consume_validate.py +++ b/tests/kafkatest/tests/produce_consume_validate.py @@ -15,6 +15,9 @@ from ducktape.tests.test import Test from ducktape.utils.util import wait_until + +from kafkatest.utils import validate_delivery + import time class ProduceConsumeValidateTest(Test): @@ -115,68 +118,21 @@ def run_produce_consume_validate(self, core_test_action=None, *args): self.mark_for_collect(s) raise - @staticmethod - def annotate_missing_msgs(missing, acked, consumed, msg): - missing_list = list(missing) - msg += "%s acked message did not make it to the Consumer. They are: " %\ - len(missing_list) - if len(missing_list) < 20: - msg += str(missing_list) + ". " - else: - msg += ", ".join(str(m) for m in missing_list[:20]) - msg += "...plus %s more. Total Acked: %s, Total Consumed: %s. " \ - % (len(missing_list) - 20, len(set(acked)), len(set(consumed))) - return msg - - @staticmethod - def annotate_data_lost(data_lost, msg, number_validated): - print_limit = 10 - if len(data_lost) > 0: - msg += "The first %s missing messages were validated to ensure they are in Kafka's data files. " \ - "%s were missing. This suggests data loss. Here are some of the messages not found in the data files: %s\n" \ - % (number_validated, len(data_lost), str(data_lost[0:print_limit]) if len(data_lost) > print_limit else str(data_lost)) - else: - msg += "We validated that the first %s of these missing messages correctly made it into Kafka's data files. " \ - "This suggests they were lost on their way to the consumer." % number_validated - return msg - def validate(self): - """Check that each acked message was consumed.""" - success = True - msg = "" - acked = self.producer.acked - consumed = self.consumer.messages_consumed[1] - # Correctness of the set difference operation depends on using equivalent message_validators in procuder and consumer - missing = set(acked) - set(consumed) - - self.logger.info("num consumed: %d" % len(consumed)) - - # Were all acked messages consumed? - if len(missing) > 0: - msg = self.annotate_missing_msgs(missing, acked, consumed, msg) - success = False - - #Did we miss anything due to data loss? - to_validate = list(missing)[0:1000 if len(missing) > 1000 else len(missing)] - data_lost = self.kafka.search_data_files(self.topic, to_validate) - msg = self.annotate_data_lost(data_lost, msg, len(to_validate)) + messages_consumed = self.consumer.messages_consumed[1] + self.logger.info("Number of acked records: %d" % len(self.producer.acked)) + self.logger.info("Number of consumed records: %d" % len(messages_consumed)) - if self.enable_idempotence: - self.logger.info("Ran a test with idempotence enabled. We expect no duplicates") - else: - self.logger.info("Ran a test with idempotence disabled.") + def check_lost_data(missing_records): + return self.kafka.search_data_files(self.topic, missing_records) - # Are there duplicates? - if len(set(consumed)) != len(consumed): - num_duplicates = abs(len(set(consumed)) - len(consumed)) - msg += "(There are also %s duplicate messages in the log - but that is an acceptable outcome)\n" % num_duplicates - if self.enable_idempotence: - assert False, "Detected %s duplicates even though idempotence was enabled." % num_duplicates + succeeded, error_msg = validate_delivery(self.producer.acked, messages_consumed, + self.enable_idempotence, check_lost_data) # Collect all logs if validation fails - if not success: + if not succeeded: for s in self.test_context.services: self.mark_for_collect(s) - assert success, msg + assert succeeded, error_msg diff --git a/tests/kafkatest/tests/streams/base_streams_test.py b/tests/kafkatest/tests/streams/base_streams_test.py index 320d4b2068b55..9a9704e3b8423 100644 --- a/tests/kafkatest/tests/streams/base_streams_test.py +++ b/tests/kafkatest/tests/streams/base_streams_test.py @@ -38,13 +38,14 @@ def get_consumer(self, client_id, topic, num_messages): client_id, max_messages=num_messages) - def get_producer(self, topic, num_messages, repeating_keys=None): + def get_producer(self, topic, num_messages, throughput=1000, repeating_keys=None): return VerifiableProducer(self.test_context, 1, self.kafka, topic, max_messages=num_messages, acks=1, + throughput=throughput, repeating_keys=repeating_keys) def assert_produce_consume(self, diff --git a/tests/kafkatest/tests/streams/streams_broker_bounce_test.py b/tests/kafkatest/tests/streams/streams_broker_bounce_test.py index 8d623eb727e5b..7859d699ee58e 100644 --- a/tests/kafkatest/tests/streams/streams_broker_bounce_test.py +++ b/tests/kafkatest/tests/streams/streams_broker_bounce_test.py @@ -133,6 +133,22 @@ def fail_many_brokers(self, failure_mode, num_failures): for num in range(0, num_failures - 1): signal_node(self, self.kafka.nodes[num], sig) + def confirm_topics_on_all_brokers(self, expected_topic_set): + for node in self.kafka.nodes: + match_count = 0 + # need to iterate over topic_list_generator as kafka.list_topics() + # returns a python generator so values are fetched lazily + # so we can't just compare directly we must iterate over what's returned + topic_list_generator = self.kafka.list_topics("placeholder", node) + for topic in topic_list_generator: + if topic in expected_topic_set: + match_count += 1 + + if len(expected_topic_set) != match_count: + return False + + return True + def setup_system(self, start_processor=True): # Setup phase @@ -141,6 +157,12 @@ def setup_system(self, start_processor=True): self.kafka = KafkaService(self.test_context, num_nodes=self.replication, zk=self.zk, topics=self.topics) self.kafka.start() + + # allow some time for topics to be created + wait_until(lambda: self.confirm_topics_on_all_brokers(set(self.topics.keys())), + timeout_sec=60, + err_msg="Broker did not create all topics in 60 seconds ") + # Start test harness self.driver = StreamsSmokeTestDriverService(self.test_context, self.kafka) self.processor1 = StreamsSmokeTestJobRunnerService(self.test_context, self.kafka) diff --git a/tests/kafkatest/tests/streams/streams_broker_down_resilience_test.py b/tests/kafkatest/tests/streams/streams_broker_down_resilience_test.py index 3cbf71390c9ba..ee5feaea1486a 100644 --- a/tests/kafkatest/tests/streams/streams_broker_down_resilience_test.py +++ b/tests/kafkatest/tests/streams/streams_broker_down_resilience_test.py @@ -27,7 +27,9 @@ class StreamsBrokerDownResilience(BaseStreamsTest): inputTopic = "streamsResilienceSource" outputTopic = "streamsResilienceSink" client_id = "streams-broker-resilience-verify-consumer" - num_messages = 5 + num_messages = 10000 + message = "processed[0-9]*messages" + connected_message = "Discovered group coordinator" def __init__(self, test_context): super(StreamsBrokerDownResilience, self).__init__(test_context, @@ -48,8 +50,6 @@ def test_streams_resilient_to_broker_down(self): processor = StreamsBrokerDownResilienceService(self.test_context, self.kafka, self.get_configs()) processor.start() - # until KIP-91 is merged we'll only send 5 messages to assert Kafka Streams is running before taking the broker down - # After KIP-91 is merged we'll continue to send messages the duration of the test self.assert_produce_consume(self.inputTopic, self.outputTopic, self.client_id, @@ -61,7 +61,11 @@ def test_streams_resilient_to_broker_down(self): time.sleep(broker_down_time_in_seconds) - self.kafka.start_node(node) + with processor.node.account.monitor_log(processor.LOG_FILE) as monitor: + self.kafka.start_node(node) + monitor.wait_until(self.connected_message, + timeout_sec=120, + err_msg=("Never saw output '%s' on " % self.connected_message) + str(processor.node.account)) self.assert_produce_consume(self.inputTopic, self.outputTopic, @@ -95,22 +99,45 @@ def test_streams_runs_with_broker_down_initially(self): self.wait_for_verification(processor_2, broker_unavailable_message, processor_2.LOG_FILE, 10) self.wait_for_verification(processor_3, broker_unavailable_message, processor_3.LOG_FILE, 10) - # now start broker - self.kafka.start_node(node) - - # assert streams can process when starting with broker down - self.assert_produce_consume(self.inputTopic, - self.outputTopic, - self.client_id, - "running_with_broker_down_initially", - num_messages=9, - timeout_sec=120) - - message = "processed3messages" - # need to show all 3 instances processed messages - self.wait_for_verification(processor, message, processor.STDOUT_FILE) - self.wait_for_verification(processor_2, message, processor_2.STDOUT_FILE) - self.wait_for_verification(processor_3, message, processor_3.STDOUT_FILE) + with processor.node.account.monitor_log(processor.LOG_FILE) as monitor_1: + with processor_2.node.account.monitor_log(processor_2.LOG_FILE) as monitor_2: + with processor_3.node.account.monitor_log(processor_3.LOG_FILE) as monitor_3: + self.kafka.start_node(node) + + monitor_1.wait_until(self.connected_message, + timeout_sec=120, + err_msg=("Never saw '%s' on " % self.connected_message) + str(processor.node.account)) + monitor_2.wait_until(self.connected_message, + timeout_sec=120, + err_msg=("Never saw '%s' on " % self.connected_message) + str(processor_2.node.account)) + monitor_3.wait_until(self.connected_message, + timeout_sec=120, + err_msg=("Never saw '%s' on " % self.connected_message) + str(processor_3.node.account)) + + with processor.node.account.monitor_log(processor.STDOUT_FILE) as monitor_1: + with processor_2.node.account.monitor_log(processor_2.STDOUT_FILE) as monitor_2: + with processor_3.node.account.monitor_log(processor_3.STDOUT_FILE) as monitor_3: + + self.assert_produce(self.inputTopic, + "sending_message_after_broker_down_initially", + num_messages=self.num_messages, + timeout_sec=120) + + monitor_1.wait_until(self.message, + timeout_sec=120, + err_msg=("Never saw '%s' on " % self.message) + str(processor.node.account)) + monitor_2.wait_until(self.message, + timeout_sec=120, + err_msg=("Never saw '%s' on " % self.message) + str(processor_2.node.account)) + monitor_3.wait_until(self.message, + timeout_sec=120, + err_msg=("Never saw '%s' on " % self.message) + str(processor_3.node.account)) + + self.assert_consume(self.client_id, + "consuming_message_after_broker_down_initially", + self.outputTopic, + num_messages=self.num_messages, + timeout_sec=120) self.kafka.stop() @@ -126,24 +153,40 @@ def test_streams_should_scale_in_while_brokers_down(self): processor_2.start() processor_3 = StreamsBrokerDownResilienceService(self.test_context, self.kafka, configs) - processor_3.start() # need to wait for rebalance once - self.wait_for_verification(processor_3, "State transition from REBALANCING to RUNNING", processor_3.LOG_FILE) - - # assert streams can process when starting with broker up - self.assert_produce_consume(self.inputTopic, - self.outputTopic, - self.client_id, - "waiting for rebalance to complete", - num_messages=9, - timeout_sec=120) - - message = "processed3messages" - - self.wait_for_verification(processor, message, processor.STDOUT_FILE) - self.wait_for_verification(processor_2, message, processor_2.STDOUT_FILE) - self.wait_for_verification(processor_3, message, processor_3.STDOUT_FILE) + rebalance = "State transition from REBALANCING to RUNNING" + with processor_3.node.account.monitor_log(processor_3.LOG_FILE) as monitor: + processor_3.start() + + monitor.wait_until(rebalance, + timeout_sec=120, + err_msg=("Never saw output '%s' on " % rebalance) + str(processor_3.node.account)) + + with processor.node.account.monitor_log(processor.STDOUT_FILE) as monitor_1: + with processor_2.node.account.monitor_log(processor_2.STDOUT_FILE) as monitor_2: + with processor_3.node.account.monitor_log(processor_3.STDOUT_FILE) as monitor_3: + + self.assert_produce(self.inputTopic, + "sending_message_normal_broker_start", + num_messages=self.num_messages, + timeout_sec=120) + + monitor_1.wait_until(self.message, + timeout_sec=120, + err_msg=("Never saw '%s' on " % self.message) + str(processor.node.account)) + monitor_2.wait_until(self.message, + timeout_sec=120, + err_msg=("Never saw '%s' on " % self.message) + str(processor_2.node.account)) + monitor_3.wait_until(self.message, + timeout_sec=120, + err_msg=("Never saw '%s' on " % self.message) + str(processor_3.node.account)) + + self.assert_consume(self.client_id, + "consuming_message_normal_broker_start", + self.outputTopic, + num_messages=self.num_messages, + timeout_sec=120) node = self.kafka.leader(self.inputTopic) self.kafka.stop_node(node) @@ -155,17 +198,20 @@ def test_streams_should_scale_in_while_brokers_down(self): self.wait_for_verification(processor, shutdown_message, processor.STDOUT_FILE) self.wait_for_verification(processor_2, shutdown_message, processor_2.STDOUT_FILE) - self.kafka.start_node(node) + with processor_3.node.account.monitor_log(processor_3.LOG_FILE) as monitor_3: + self.kafka.start_node(node) + + monitor_3.wait_until(self.connected_message, + timeout_sec=120, + err_msg=("Never saw '%s' on " % self.connected_message) + str(processor_3.node.account)) self.assert_produce_consume(self.inputTopic, self.outputTopic, self.client_id, "sending_message_after_stopping_streams_instance_bouncing_broker", - num_messages=9, + num_messages=self.num_messages, timeout_sec=120) - self.wait_for_verification(processor_3, "processed9messages", processor_3.STDOUT_FILE) - self.kafka.stop() def test_streams_should_failover_while_brokers_down(self): @@ -180,24 +226,40 @@ def test_streams_should_failover_while_brokers_down(self): processor_2.start() processor_3 = StreamsBrokerDownResilienceService(self.test_context, self.kafka, configs) - processor_3.start() # need to wait for rebalance once - self.wait_for_verification(processor_3, "State transition from REBALANCING to RUNNING", processor_3.LOG_FILE) - - # assert streams can process when starting with broker up - self.assert_produce_consume(self.inputTopic, - self.outputTopic, - self.client_id, - "waiting for rebalance to complete", - num_messages=9, - timeout_sec=120) - - message = "processed3messages" - - self.wait_for_verification(processor, message, processor.STDOUT_FILE) - self.wait_for_verification(processor_2, message, processor_2.STDOUT_FILE) - self.wait_for_verification(processor_3, message, processor_3.STDOUT_FILE) + rebalance = "State transition from REBALANCING to RUNNING" + with processor_3.node.account.monitor_log(processor_3.LOG_FILE) as monitor: + processor_3.start() + + monitor.wait_until(rebalance, + timeout_sec=120, + err_msg=("Never saw output '%s' on " % rebalance) + str(processor_3.node.account)) + + with processor.node.account.monitor_log(processor.STDOUT_FILE) as monitor_1: + with processor_2.node.account.monitor_log(processor_2.STDOUT_FILE) as monitor_2: + with processor_3.node.account.monitor_log(processor_3.STDOUT_FILE) as monitor_3: + + self.assert_produce(self.inputTopic, + "sending_message_after_normal_broker_start", + num_messages=self.num_messages, + timeout_sec=120) + + monitor_1.wait_until(self.message, + timeout_sec=120, + err_msg=("Never saw '%s' on " % self.message) + str(processor.node.account)) + monitor_2.wait_until(self.message, + timeout_sec=120, + err_msg=("Never saw '%s' on " % self.message) + str(processor_2.node.account)) + monitor_3.wait_until(self.message, + timeout_sec=120, + err_msg=("Never saw '%s' on " % self.message) + str(processor_3.node.account)) + + self.assert_consume(self.client_id, + "consuming_message_after_normal_broker_start", + self.outputTopic, + num_messages=self.num_messages, + timeout_sec=120) node = self.kafka.leader(self.inputTopic) self.kafka.stop_node(node) @@ -206,13 +268,43 @@ def test_streams_should_failover_while_brokers_down(self): processor_2.abortThenRestart() processor_3.abortThenRestart() - self.kafka.start_node(node) - - self.assert_produce_consume(self.inputTopic, - self.outputTopic, - self.client_id, - "sending_message_after_hard_bouncing_streams_instance_bouncing_broker", - num_messages=9, - timeout_sec=120) - + with processor.node.account.monitor_log(processor.LOG_FILE) as monitor_1: + with processor_2.node.account.monitor_log(processor_2.LOG_FILE) as monitor_2: + with processor_3.node.account.monitor_log(processor_3.LOG_FILE) as monitor_3: + self.kafka.start_node(node) + + monitor_1.wait_until(self.connected_message, + timeout_sec=120, + err_msg=("Never saw '%s' on " % self.connected_message) + str(processor.node.account)) + monitor_2.wait_until(self.connected_message, + timeout_sec=120, + err_msg=("Never saw '%s' on " % self.connected_message) + str(processor_2.node.account)) + monitor_3.wait_until(self.connected_message, + timeout_sec=120, + err_msg=("Never saw '%s' on " % self.connected_message) + str(processor_3.node.account)) + + with processor.node.account.monitor_log(processor.STDOUT_FILE) as monitor_1: + with processor_2.node.account.monitor_log(processor_2.STDOUT_FILE) as monitor_2: + with processor_3.node.account.monitor_log(processor_3.STDOUT_FILE) as monitor_3: + + self.assert_produce(self.inputTopic, + "sending_message_after_hard_bouncing_streams_instance_bouncing_broker", + num_messages=self.num_messages, + timeout_sec=120) + + monitor_1.wait_until(self.message, + timeout_sec=120, + err_msg=("Never saw '%s' on " % self.message) + str(processor.node.account)) + monitor_2.wait_until(self.message, + timeout_sec=120, + err_msg=("Never saw '%s' on " % self.message) + str(processor_2.node.account)) + monitor_3.wait_until(self.message, + timeout_sec=120, + err_msg=("Never saw '%s' on " % self.message) + str(processor_3.node.account)) + + self.assert_consume(self.client_id, + "consuming_message_after_stopping_streams_instance_bouncing_broker", + self.outputTopic, + num_messages=self.num_messages, + timeout_sec=120) self.kafka.stop() diff --git a/tests/kafkatest/tests/streams/streams_standby_replica_test.py b/tests/kafkatest/tests/streams/streams_standby_replica_test.py index 416a110f0649c..8425e14c0bf06 100644 --- a/tests/kafkatest/tests/streams/streams_standby_replica_test.py +++ b/tests/kafkatest/tests/streams/streams_standby_replica_test.py @@ -46,7 +46,7 @@ def test_standby_tasks_rebalance(self): self.streams_sink_topic_1, self.streams_sink_topic_2)) - producer = self.get_producer(self.streams_source_topic, self.num_messages, repeating_keys=6) + producer = self.get_producer(self.streams_source_topic, self.num_messages, throughput=15000, repeating_keys=6) producer.start() processor_1 = StreamsStandbyTaskService(self.test_context, self.kafka, configs) diff --git a/tests/kafkatest/tests/streams/streams_upgrade_test.py b/tests/kafkatest/tests/streams/streams_upgrade_test.py index 41134672e9871..3e68e205b92ab 100644 --- a/tests/kafkatest/tests/streams/streams_upgrade_test.py +++ b/tests/kafkatest/tests/streams/streams_upgrade_test.py @@ -15,7 +15,7 @@ import random import time -from ducktape.mark import ignore, matrix +from ducktape.mark import matrix, ignore from ducktape.mark.resource import cluster from ducktape.tests.test import Test from kafkatest.services.kafka import KafkaService @@ -28,10 +28,10 @@ metadata_1_versions = [str(LATEST_0_10_0)] metadata_2_versions = [str(LATEST_0_10_1), str(LATEST_0_10_2), str(LATEST_0_11_0), str(LATEST_1_0), str(LATEST_1_1)] -# we can add the following versions to `backward_compatible_metadata_2_versions` after the corresponding -# bug-fix release 0.10.1.2, 0.10.2.2, 0.11.0.3, 1.0.2, and 1.1.1 are available: -# str(LATEST_0_10_1), str(LATEST_0_10_2), str(LATEST_0_11_0), str(LATEST_1_0), str(LATEST_1_1) -backward_compatible_metadata_2_versions = [] +# once 0.10.0.1.2 is released we can replace +# backward_compatible_metadata_2_versions with metadata_2_versions + +backward_compatible_metadata_2_versions = [str(LATEST_0_10_2), str(LATEST_0_11_0), str(LATEST_1_0), str(LATEST_1_1)] metadata_3_versions = [str(DEV_VERSION)] class StreamsUpgradeTest(Test): @@ -125,7 +125,6 @@ def test_upgrade_downgrade_brokers(self, from_version, to_version): node.account.ssh("grep ALL-RECORDS-DELIVERED %s" % self.driver.STDOUT_FILE, allow_fail=False) self.processor1.node.account.ssh_capture("grep SMOKE-TEST-CLIENT-CLOSED %s" % self.processor1.STDOUT_FILE, allow_fail=False) - @ignore @matrix(from_version=metadata_2_versions, to_version=metadata_2_versions) def test_simple_upgrade_downgrade(self, from_version, to_version): """ diff --git a/tests/kafkatest/tests/tools/log4j_appender_test.py b/tests/kafkatest/tests/tools/log4j_appender_test.py index 7e0b9eeccecd6..6c268da839321 100644 --- a/tests/kafkatest/tests/tools/log4j_appender_test.py +++ b/tests/kafkatest/tests/tools/log4j_appender_test.py @@ -64,10 +64,9 @@ def custom_message_validator(self, msg): self.logger.debug("Received message: %s" % msg) self.messages_received_count += 1 - def start_consumer(self, security_protocol): - enable_new_consumer = security_protocol != SecurityConfig.PLAINTEXT + def start_consumer(self): self.consumer = ConsoleConsumer(self.test_context, num_nodes=self.num_brokers, kafka=self.kafka, topic=TOPIC, - consumer_timeout_ms=1000, new_consumer=enable_new_consumer, + consumer_timeout_ms=10000, message_validator=self.custom_message_validator) self.consumer.start() @@ -84,7 +83,7 @@ def test_log4j_appender(self, security_protocol='PLAINTEXT'): self.start_appender(security_protocol) self.appender.wait() - self.start_consumer(security_protocol) + self.start_consumer() node = self.consumer.nodes[0] wait_until(lambda: self.consumer.alive(node), diff --git a/tests/kafkatest/utils/__init__.py b/tests/kafkatest/utils/__init__.py index 8c473bfa63590..1c1d5e02a5983 100644 --- a/tests/kafkatest/utils/__init__.py +++ b/tests/kafkatest/utils/__init__.py @@ -13,4 +13,4 @@ # See the License for the specific language governing permissions and # limitations under the License. -from util import kafkatest_version, is_version, is_int, is_int_with_prefix, node_is_reachable +from util import kafkatest_version, is_version, is_int, is_int_with_prefix, node_is_reachable, validate_delivery diff --git a/tests/kafkatest/utils/remote_account.py b/tests/kafkatest/utils/remote_account.py index b69a59180f613..a18215b737d56 100644 --- a/tests/kafkatest/utils/remote_account.py +++ b/tests/kafkatest/utils/remote_account.py @@ -22,6 +22,13 @@ def file_exists(node, file): except: return False +def path_exists(node, path): + """Quick and dirty check for existence of remote path.""" + try: + node.account.ssh("ls " + path, allow_fail=False) + return True + except: + return False def line_count(node, file): """Return the line count of file on node""" diff --git a/tests/kafkatest/utils/util.py b/tests/kafkatest/utils/util.py index dd2027331dd70..b9ccaf84f2fe6 100644 --- a/tests/kafkatest/utils/util.py +++ b/tests/kafkatest/utils/util.py @@ -112,3 +112,59 @@ def node_is_reachable(src_node, dst_node): :return: True only if dst is reachable from src. """ return 0 == src_node.account.ssh("nc -w 3 -z %s 22" % dst_node.account.hostname, allow_fail=True) + + +def annotate_missing_msgs(missing, acked, consumed, msg): + missing_list = list(missing) + msg += "%s acked message did not make it to the Consumer. They are: " %\ + len(missing_list) + if len(missing_list) < 20: + msg += str(missing_list) + ". " + else: + msg += ", ".join(str(m) for m in missing_list[:20]) + msg += "...plus %s more. Total Acked: %s, Total Consumed: %s. " \ + % (len(missing_list) - 20, len(set(acked)), len(set(consumed))) + return msg + +def annotate_data_lost(data_lost, msg, number_validated): + print_limit = 10 + if len(data_lost) > 0: + msg += "The first %s missing messages were validated to ensure they are in Kafka's data files. " \ + "%s were missing. This suggests data loss. Here are some of the messages not found in the data files: %s\n" \ + % (number_validated, len(data_lost), str(data_lost[0:print_limit]) if len(data_lost) > print_limit else str(data_lost)) + else: + msg += "We validated that the first %s of these missing messages correctly made it into Kafka's data files. " \ + "This suggests they were lost on their way to the consumer." % number_validated + return msg + +def validate_delivery(acked, consumed, idempotence_enabled=False, check_lost_data=None): + """Check that each acked message was consumed.""" + success = True + msg = "" + + # Correctness of the set difference operation depends on using equivalent + # message_validators in producer and consumer + missing = set(acked) - set(consumed) + + # Were all acked messages consumed? + if len(missing) > 0: + msg = annotate_missing_msgs(missing, acked, consumed, msg) + success = False + + # Did we miss anything due to data loss? + if check_lost_data: + to_validate = list(missing)[0:1000 if len(missing) > 1000 else len(missing)] + data_lost = check_lost_data(to_validate) + msg = annotate_data_lost(data_lost, msg, len(to_validate)) + + # Are there duplicates? + if len(set(consumed)) != len(consumed): + num_duplicates = abs(len(set(consumed)) - len(consumed)) + + if idempotence_enabled: + success = False + msg += "Detected %d duplicates even though idempotence was enabled.\n" % num_duplicates + else: + msg += "(There are also %d duplicate messages in the log - but that is an acceptable outcome)\n" % num_duplicates + + return success, msg diff --git a/tests/kafkatest/version.py b/tests/kafkatest/version.py index 0ed29a34968a0..896b150f880c5 100644 --- a/tests/kafkatest/version.py +++ b/tests/kafkatest/version.py @@ -14,9 +14,8 @@ # limitations under the License. -from kafkatest.utils import kafkatest_version - from distutils.version import LooseVersion +from kafkatest.utils import kafkatest_version class KafkaVersion(LooseVersion): @@ -61,7 +60,7 @@ def get_version(node=None): return DEV_BRANCH DEV_BRANCH = KafkaVersion("dev") -DEV_VERSION = KafkaVersion("2.0.0-SNAPSHOT") +DEV_VERSION = KafkaVersion("2.0.2-SNAPSHOT") # 0.8.2.x versions V_0_8_2_1 = KafkaVersion("0.8.2.1") @@ -86,7 +85,8 @@ def get_version(node=None): # 0.10.2.x versions V_0_10_2_0 = KafkaVersion("0.10.2.0") V_0_10_2_1 = KafkaVersion("0.10.2.1") -LATEST_0_10_2 = V_0_10_2_1 +V_0_10_2_2 = KafkaVersion("0.10.2.2") +LATEST_0_10_2 = V_0_10_2_2 LATEST_0_10 = LATEST_0_10_2 @@ -94,14 +94,21 @@ def get_version(node=None): V_0_11_0_0 = KafkaVersion("0.11.0.0") V_0_11_0_1 = KafkaVersion("0.11.0.1") V_0_11_0_2 = KafkaVersion("0.11.0.2") -LATEST_0_11_0 = V_0_11_0_2 +V_0_11_0_3 = KafkaVersion("0.11.0.3") +LATEST_0_11_0 = V_0_11_0_3 LATEST_0_11 = LATEST_0_11_0 # 1.0.x versions V_1_0_0 = KafkaVersion("1.0.0") V_1_0_1 = KafkaVersion("1.0.1") -LATEST_1_0 = V_1_0_1 +V_1_0_2 = KafkaVersion("1.0.2") +LATEST_1_0 = V_1_0_2 # 1.1.x versions V_1_1_0 = KafkaVersion("1.1.0") -LATEST_1_1 = V_1_1_0 +V_1_1_1 = KafkaVersion("1.1.1") +LATEST_1_1 = V_1_1_1 + +# 2.0.x versions +V_2_0_0 = KafkaVersion("2.0.0") +LATEST_2_0 = V_2_0_0 diff --git a/tests/setup.py b/tests/setup.py index 7d7c4a4605af5..a0de1d4c6ab1a 100644 --- a/tests/setup.py +++ b/tests/setup.py @@ -51,7 +51,7 @@ def run_tests(self): license="apache2.0", packages=find_packages(), include_package_data=True, - install_requires=["ducktape==0.7.1", "requests>=2.5.0"], + install_requires=["ducktape==0.7.5", "requests==2.20.0"], tests_require=["pytest", "mock"], cmdclass={'test': PyTest}, ) diff --git a/vagrant/aws/aws-example-Vagrantfile.local b/vagrant/aws/aws-example-Vagrantfile.local index ee9db9a616ba6..23187a085d3df 100644 --- a/vagrant/aws/aws-example-Vagrantfile.local +++ b/vagrant/aws/aws-example-Vagrantfile.local @@ -17,6 +17,7 @@ # To use it, move it to the base kafka directory and rename # it to Vagrantfile.local, and adjust variables as needed. ec2_instance_type = "m3.xlarge" +ec2_spot_max_price = "0.266" # On-demand price for instance type enable_hostmanager = false num_zookeepers = 0 num_brokers = 0 diff --git a/vagrant/aws/aws-init.sh b/vagrant/aws/aws-init.sh index 75176263438bf..54092c82a29a6 100755 --- a/vagrant/aws/aws-init.sh +++ b/vagrant/aws/aws-init.sh @@ -31,15 +31,18 @@ base_dir=`dirname $0`/../.. if [ -z `which vagrant` ]; then echo "Installing vagrant..." - wget https://releases.hashicorp.com/vagrant/1.9.3/vagrant_1.9.3_x86_64.deb - sudo dpkg -i vagrant_1.9.3_x86_64.deb - rm -f vagrant_1.9.3_x86_64.deb + wget https://releases.hashicorp.com/vagrant/2.1.5/vagrant_2.1.5_x86_64.deb + sudo dpkg -i vagrant_2.1.5_x86_64.deb + rm -f vagrant_2.1.5_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" + +# Custom vagrant-aws with spot instance support. See https://github.com/mitchellh/vagrant-aws/issues/32 +wget -nv https://s3-us-west-2.amazonaws.com/confluent-packaging-tools/vagrant-aws-0.7.2.spot.gem -P /tmp +vagrant_plugins="/tmp/vagrant-aws-0.7.2.spot.gem vagrant-hostmanager" existing=`vagrant plugin list` for plugin in $vagrant_plugins; do echo $existing | grep $plugin > /dev/null diff --git a/vagrant/base.sh b/vagrant/base.sh index c16225d056809..3068c228f1293 100755 --- a/vagrant/base.sh +++ b/vagrant/base.sh @@ -20,38 +20,45 @@ set -ex # If you update this, also update tests/docker/Dockerfile export KIBOSH_VERSION=d85ac3ec44be0700efe605c16289fd901cfdaa13 -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 +path_to_jdk_cache() { + jdk_version=$1 + echo "/tmp/jdk-${jdk_version}.tar.gz" +} - # Try to share cache. See Vagrantfile for details - mkdir -p /var/cache/oracle-jdk8-installer - if [ -e "/tmp/oracle-jdk8-installer-cache/" ]; then - find /tmp/oracle-jdk8-installer-cache/ -not -empty -exec cp '{}' /var/cache/oracle-jdk8-installer/ \; - fi - if [ ! -e "/var/cache/oracle-jdk8-installer/jdk-8u171-linux-x64.tar.gz" ]; then - # Grab a copy of the JDK since it has moved and original downloader won't work - curl -s -L "https://s3-us-west-2.amazonaws.com/kafka-packages/jdk-8u171-linux-x64.tar.gz" -o /var/cache/oracle-jdk8-installer/jdk-8u171-linux-x64.tar.gz - fi +fetch_jdk_tgz() { + jdk_version=$1 - /bin/echo debconf shared/accepted-oracle-license-v1-1 select true | /usr/bin/debconf-set-selections + path=$(path_to_jdk_cache $jdk_version) - # oracle-javaX-installer runs wget with a dot progress indicator which ends up - # as one line per dot in the build logs. - # To avoid this noise we redirect all output to a file that we only show if apt-get fails. - echo "Installing JDK..." - if ! apt-get -y install oracle-java8-installer oracle-java8-set-default >/tmp/jdk_install.log 2>&1 ; then - cat /tmp/jdk_install.log - echo "ERROR: JDK install failed" - exit 1 - fi + if [ ! -e $path ]; then + mkdir -p $(dirname $path) + curl -s -L "https://s3-us-west-2.amazonaws.com/kafka-packages/jdk-${jdk_version}.tar.gz" -o $path + fi +} + +JDK_MAJOR="${JDK_MAJOR:-8}" +JDK_FULL="${JDK_FULL:-8u202-linux-x64}" + +if [ -z `which javac` ]; then + apt-get -y update + apt-get install -y software-properties-common python-software-properties binutils java-common + + echo "===> Installing JDK..." + + mkdir -p /opt/jdk + cd /opt/jdk + rm -rf $JDK_MAJOR + mkdir -p $JDK_MAJOR + cd $JDK_MAJOR + fetch_jdk_tgz $JDK_FULL + tar x --strip-components=1 -zf $(path_to_jdk_cache $JDK_FULL) + for bin in /opt/jdk/$JDK_MAJOR/bin/* ; do + name=$(basename $bin) + update-alternatives --install /usr/bin/$name $name $bin 1081 && update-alternatives --set $name $bin + done + echo -e "export JAVA_HOME=/opt/jdk/$JDK_MAJOR\nexport PATH=\$PATH:\$JAVA_HOME/bin" > /etc/profile.d/jdk.sh echo "JDK installed: $(javac -version 2>&1)" - if [ -e "/tmp/oracle-jdk8-installer-cache/" ]; then - cp -R /var/cache/oracle-jdk8-installer/* /tmp/oracle-jdk8-installer-cache - fi fi chmod a+rw /opt @@ -111,14 +118,14 @@ get_kafka 0.10.0.1 2.11 chmod a+rw /opt/kafka-0.10.0.1 get_kafka 0.10.1.1 2.11 chmod a+rw /opt/kafka-0.10.1.1 -get_kafka 0.10.2.1 2.11 -chmod a+rw /opt/kafka-0.10.2.1 -get_kafka 0.11.0.2 2.11 -chmod a+rw /opt/kafka-0.11.0.2 -get_kafka 1.0.1 2.11 -chmod a+rw /opt/kafka-1.0.1 -get_kafka 1.1.0 2.11 -chmod a+rw /opt/kafka-1.1.0 +get_kafka 0.10.2.2 2.11 +chmod a+rw /opt/kafka-0.10.2.2 +get_kafka 0.11.0.3 2.11 +chmod a+rw /opt/kafka-0.11.0.3 +get_kafka 1.0.2 2.11 +chmod a+rw /opt/kafka-1.0.2 +get_kafka 1.1.1 2.11 +chmod a+rw /opt/kafka-1.1.1 # For EC2 nodes, we want to use /mnt, which should have the local disk. On local diff --git a/wrapper.gradle b/wrapper.gradle index bc6350632e2a6..3ce451ab4f13d 100644 --- a/wrapper.gradle +++ b/wrapper.gradle @@ -17,9 +17,108 @@ * under the License. */ -defaultTasks 'downloadWrapper' +// This file contains tasks for the gradle wrapper generation. -task downloadWrapper(type: Wrapper) { - description = "Download the gradle wrapper and requisite files. Overwrites existing wrapper files." +// Ensure the wrapper script is generated based on the version defined in the project +// and not the version installed on the machine running the task. +// Read more about the wrapper here: https://docs.gradle.org/current/userguide/gradle_wrapper.html +wrapper { gradleVersion = project.gradleVersion -} \ No newline at end of file + distributionType = Wrapper.DistributionType.ALL +} + +def licenseString = """# +# Copyright 2017 the original author or authors. +# +# 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.""" + +// Custom task to inject support for downloading the gradle wrapper jar if it doesn't exist. +// This allows us to avoid checking in the jar to our repository. +// Additionally adds a license header to the wrapper while editing the file contents. +task bootstrapWrapper() { + // In the doLast block so this runs when the task is called and not during project configuration. + doLast { + def wrapperBasePath = "\$APP_HOME/gradle/wrapper" + def wrapperJarPath = wrapperBasePath + "/gradle-wrapper.jar" + + // Add a trailing zero to the version if needed. + def fullVersion = project.gradleVersion.count(".") == 1 ? "${project.gradleVersion}.0" : versions.gradle + // Leverages the wrapper jar checked into the gradle project on github because the jar isn't + // available elsewhere. Using raw.githubusercontent.com instead of github.com because + // github.com servers deprecated TLSv1/TLSv1.1 support some time ago, so older versions + // of curl (built against OpenSSL library that doesn't support TLSv1.2) would fail to + // fetch the jar. + def wrapperBaseUrl = "https://raw.githubusercontent.com/gradle/gradle/v$fullVersion/gradle/wrapper" + def wrapperJarUrl = wrapperBaseUrl + "/gradle-wrapper.jar" + + def bootstrapString = """ + # Loop in case we encounter an error. + for attempt in 1 2 3; do + if [ ! -e $wrapperJarPath ]; then + if ! curl -s -S --retry 3 -L -o "$wrapperJarPath" "$wrapperJarUrl"; then + rm -f "$wrapperJarPath" + # Pause for a bit before looping in case the server throttled us. + sleep 5 + continue + fi + fi + done + """.stripIndent() + + def wrapperScript = wrapper.scriptFile + def wrapperLines = wrapperScript.readLines() + wrapperScript.withPrintWriter { out -> + def licenseWritten = false + def bootstrapWritten = false + wrapperLines.each { line -> + // Print the wrapper bootstrap before the first usage of the wrapper jar. + if (!bootstrapWritten && line.contains("gradle-wrapper.jar")) { + out.println(bootstrapString) + bootstrapWritten = true + } + out.print(line) + // Print the licence after the shebang. + if(!licenseWritten && line.contains("#!/usr/bin/env sh")) { + out.println() + out.print(licenseString) + licenseWritten = true + } + out.println() // New Line + } + } + } +} +wrapper.finalizedBy bootstrapWrapper + +// Custom task to add a license header to the gradle-wrapper.properties file. +task bootstrapWrapperProperties() { + // In the doLast block so this runs when the task is called and not during project configuration. + doLast { + def wrapperProperties = wrapper.propertiesFile + def wrapperLines = wrapperProperties.readLines() + wrapperProperties.withPrintWriter { out -> + // Print the license + out.println(licenseString) + wrapperLines.each { line -> + out.println(line) + } + } + } +} +wrapper.finalizedBy bootstrapWrapperProperties + +// Remove the generated batch file since we don't test building in the Windows environment. +task removeWindowsScript(type: Delete) { + delete "$rootDir/gradlew.bat" +} +wrapper.finalizedBy removeWindowsScript